From 26543e10f8143dbc2d313b870081d633baf4cd05 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 14 Aug 2018 14:21:13 +0300 Subject: [PATCH 01/95] IGNITE-9141 Implemented --- .../messages/GridQueryNextPageResponse.java | 34 +- .../h2/twostep/GridMapQueryExecutor.java | 104 ++-- .../h2/twostep/GridReduceQueryExecutor.java | 20 +- .../query/h2/twostep/ReduceQueryRun.java | 21 + .../dht/MockGridDhtLocalPartition.java | 310 ++++++++++ .../h2/twostep/RetryCauseMessageTest.java | 560 ++++++++++++++++++ 6 files changed, 999 insertions(+), 50 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java index 4d918a04eacb8..925823c571c87 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java @@ -67,6 +67,9 @@ public class GridQueryNextPageResponse implements Message { /** */ private AffinityTopologyVersion retry; + /** Retry cause description*/ + private String retryCause; + /** Last page flag. */ private boolean last; @@ -230,6 +233,12 @@ public Collection plainRows() { return false; writer.incrementState(); + + case 9: + if (!writer.writeString("retryCause", retryCause)) + return false; + + writer.incrementState(); } return true; @@ -314,6 +323,15 @@ public Collection plainRows() { return false; reader.incrementState(); + + case 9: + retryCause = reader.readString("retryCause"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridQueryNextPageResponse.class); @@ -326,7 +344,7 @@ public Collection plainRows() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 9; + return 10; } /** @@ -343,6 +361,18 @@ public void retry(AffinityTopologyVersion retry) { this.retry = retry; } + /** + * @return Retry Ccause message. + */ + public String retryCause() { return retryCause; } + + /** + * @param retryCause Retry Ccause message. + */ + public void retryCause(String retryCause){ + this.retryCause = retryCause; + } + /** * @return Last page flag. */ @@ -363,4 +393,4 @@ public void last(boolean last) { "valsSize", vals != null ? vals.size() : 0, "rowsSize", plainRows != null ? plainRows.size() : 0); } -} \ No newline at end of file +} 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 216a259da73e4..1af66bbea5fb0 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 @@ -296,10 +296,10 @@ private GridDhtLocalPartition partition(GridCacheContext cctx, int p) { * @param reserved Reserved list. * @param nodeId Node ID. * @param reqId Request ID. - * @return {@code true} If all the needed partitions successfully reserved. + * @return String which is null in case of success or with causeMessage if failed * @throws IgniteCheckedException If failed. */ - private boolean reservePartitions( + private String reservePartitions( @Nullable List cacheIds, AffinityTopologyVersion topVer, final int[] explicitParts, @@ -310,7 +310,7 @@ private boolean reservePartitions( assert topVer != null; if (F.isEmpty(cacheIds)) - return true; + return null; Collection partIds = wrap(explicitParts); @@ -319,11 +319,10 @@ private boolean reservePartitions( // Cache was not found, probably was not deployed yet. if (cctx == null) { - logRetry("Failed to reserve partitions for query (cache is not found on local node) [" + - "rmtNodeId=" + nodeId + ", reqId=" + reqId + ", affTopVer=" + topVer + ", cacheId=" + - cacheIds.get(i) + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query (cache is not found on local node) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", nodeId, reqId, topVer, cacheIds.get(i)); + logRetry(rslt); + return rslt; } if (cctx.isLocal() || !cctx.rebalanceEnabled()) @@ -337,11 +336,10 @@ private boolean reservePartitions( if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits. if (r != MapReplicatedReservation.INSTANCE) { if (!r.reserve()) { - logRetry("Failed to reserve partitions for query (group reservation failed) [" + - "rmtNodeId=" + nodeId + ", reqId=" + reqId + ", affTopVer=" + topVer + - ", cacheId=" + cacheIds.get(i) + ", cacheName=" + cctx.name() + "]"); - - return false; // We need explicit partitions here -> retry. + final String rslt = String.format("Failed to reserve partitions for query (group reservation failed) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s]", nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); + logRetry(rslt); + return rslt; } reserved.add(r); @@ -359,13 +357,20 @@ private boolean reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING) { - logRetry("Failed to reserve partitions for query (partition of " + - "REPLICATED cache is not in OWNING state) [rmtNodeId=" + nodeId + - ", reqId=" + reqId + ", affTopVer=" + topVer + ", cacheId=" + cacheIds.get(i) + - ", cacheName=" + cctx.name() + ", part=" + p + ", partFound=" + (part != null) + - ", partState=" + partState + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query " + + "(partition of REPLICATED cache is not in OWNING state) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + p, + (part != null), + partState + ); + logRetry(rslt); + return rslt; } } @@ -383,13 +388,20 @@ private boolean reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING || !part.reserve()) { - logRetry("Failed to reserve partitions for query (partition of " + - "PARTITIONED cache cannot be reserved) [rmtNodeId=" + nodeId + ", reqId=" + reqId + - ", affTopVer=" + topVer + ", cacheId=" + cacheIds.get(i) + - ", cacheName=" + cctx.name() + ", part=" + partId + ", partFound=" + (part != null) + - ", partState=" + partState + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query " + + "(partition of PARTITIONED cache cannot be reserved) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + partId, + (part != null), + partState + ); + logRetry(rslt); + return rslt; } reserved.add(part); @@ -398,12 +410,19 @@ private boolean reservePartitions( partState = part.state(); if (part.state() != OWNING) { - logRetry("Failed to reserve partitions for query (partition of " + - "PARTITIONED cache is not in OWNING state after reservation) [rmtNodeId=" + nodeId + - ", reqId=" + reqId + ", affTopVer=" + topVer + ", cacheId=" + cacheIds.get(i) + - ", cacheName=" + cctx.name() + ", part=" + partId + ", partState=" + partState + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query " + + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partState=%s]", + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + partId, + partState + ); + logRetry(rslt); + return rslt; } } @@ -426,7 +445,7 @@ private boolean reservePartitions( } } - return true; + return null; } /** @@ -673,12 +692,13 @@ private void onQueryRequest0( try { if (topVer != null) { // Reserve primary for topology version or explicit partitions. - if (!reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId)) { + String reservationError = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); + if (!F.isEmpty(reservationError)) { // Unregister lazy worker because re-try may never reach this node again. if (lazy) stopAndUnregisterCurrentLazyWorker(); - sendRetry(node, reqId, segmentId); + sendRetry(node, reqId, segmentId, reservationError); return; } @@ -793,10 +813,9 @@ private void onQueryRequest0( GridH2RetryException retryErr = X.cause(e, GridH2RetryException.class); if (retryErr != null) { - logRetry("Failed to execute non-collocated query (will retry) [nodeId=" + node.id() + - ", reqId=" + reqId + ", errMsg=" + retryErr.getMessage() + ']'); - - sendRetry(node, reqId, segmentId); + final String retryCause = String.format("Failed to execute non-collocated query (will retry) [nodeId=%s, reqId=%s, errMsg=%s]",node.id(),reqId,retryErr.getMessage()); + logRetry(retryCause); + sendRetry(node, reqId, segmentId,retryCause); } else { U.error(log, "Failed to execute local query.", e); @@ -845,7 +864,7 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th List reserved = new ArrayList<>(); - if (!reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId)) { + if (!F.isEmpty(reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId))) { U.error(log, "Failed to reserve partitions for DML request. [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", reqId=" + req.requestId() + ", cacheIds=" + cacheIds + ", topVer=" + topVer + ", parts=" + Arrays.toString(parts) + ']'); @@ -1081,7 +1100,7 @@ private void sendNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryRes * @param reqId Request ID. * @param segmentId Index segment ID. */ - private void sendRetry(ClusterNode node, long reqId, int segmentId) { + private void sendRetry(ClusterNode node, long reqId, int segmentId, String retryCause) { try { boolean loc = node.isLocal(); @@ -1092,6 +1111,7 @@ private void sendRetry(ClusterNode node, long reqId, int segmentId) { false); msg.retry(h2.readyTopologyVersion()); + msg.retryCause(retryCause); if (loc) h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg); 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 65aeae927cdf0..fd9ca676da563 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 @@ -38,6 +38,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import javax.cache.CacheException; @@ -289,7 +290,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { * @param node Node. * @param msg Message. */ - private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { + private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse msg) { final long qryReqId = msg.queryRequestId(); final int qry = msg.query(); final int seg = msg.segmentId(); @@ -316,7 +317,8 @@ private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { if (err0 != null && err0.getCause() instanceof IgniteClientDisconnectedException) throw err0; - CacheException e = new CacheException("Failed to fetch data from node: " + node.id()); + CacheException e = new CacheException( + (msg.retryCause()!=null) ? msg.retryCause() : "Failed to fetch data from node: " + node.id()); if (err0 != null) e.addSuppressed(err0); @@ -349,7 +351,7 @@ private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { idx.addPage(page); if (msg.retry() != null) - retry(r, msg.retry(), node.id()); + r.stateWithMsg(msg, node.id()); else if (msg.page() == 0) // Do count down on each first page received. r.latch().countDown(); } @@ -572,10 +574,13 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); - for (int attempt = 0;; attempt++) { - if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) - throw new CacheException("Failed to map SQL query to topology."); + final AtomicReference rootCause = new AtomicReference<>(); + for (int attempt = 0;; attempt++) { + if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) { + String rcValue = rootCause.get(); + throw new CacheException((!F.isEmpty(rcValue))?rcValue:("Failed to map SQL query to topology.")); + } if (attempt != 0) { try { Thread.sleep(attempt * 10); // Wait for exchange. @@ -843,6 +848,9 @@ public Iterator> query( } if (retry) { + assert r != null; + if (!F.isEmpty(r.rootCause())) + rootCause.compareAndSet(null, r.rootCause()); if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 73bb002ee1635..0097048bab935 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse; import org.h2.jdbc.JdbcConnection; import org.jetbrains.annotations.Nullable; @@ -56,6 +57,9 @@ class ReduceQueryRun { /** Can be either CacheException in case of error or AffinityTopologyVersion to retry if needed. */ private final AtomicReference state = new AtomicReference<>(); + /** Holder of root cause description*/ + private final AtomicReference rootCause = new AtomicReference<>(); + /** * Constructor. * @@ -99,6 +103,16 @@ void state(Object o, @Nullable UUID nodeId) { idx.fail(nodeId, e); } + /** + * @param msg corresponding response message + * @param nodeId Node ID. + */ + void stateWithMsg(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { + assert msg != null; + rootCause.compareAndSet(null, msg.retryCause()); + state(msg.retry(), nodeId); + } + /** * @param e Error. */ @@ -134,6 +148,13 @@ Object state() { return state.get(); } + /** + * @return Root Cause. + */ + String rootCause() { + return rootCause.get(); + } + /** * @return Indexes. */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java new file mode 100644 index 0000000000000..33d7613c9c99a --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java @@ -0,0 +1,310 @@ +package org.apache.ignite.internal.processors.cache.distributed.dht; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteInClosure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Wrapper around GridDhtLocalPartition to be extended in test cases + */ +public abstract class MockGridDhtLocalPartition extends GridDhtLocalPartition{ + /** + * fake id generator in order to bypass construction stage failure (if invoked with real id) + */ + private static AtomicInteger cntr = new AtomicInteger(Integer.MAX_VALUE); + + /** + * Real object + */ + private GridDhtLocalPartition internal; + + /** + * @param ctx Context. + * @param grp Cache group. + * @param id Partition ID. + */ + private MockGridDhtLocalPartition(GridCacheSharedContext ctx, + CacheGroupContext grp, int id) { + super(ctx, grp, id); + } + + /** */ + protected MockGridDhtLocalPartition(GridCacheSharedContext ctx, + CacheGroupContext grp, GridDhtLocalPartition internal){ + this(ctx, grp, cntr.getAndDecrement()); + this.internal = internal; + } + + /** */ + protected GridDhtLocalPartition getInternal(){ + return internal; + } + + /** {@inheritDoc} */ + @Override public int internalSize() { + return internal.internalSize(); + } + + /** {@inheritDoc} */ + @Override protected CacheMapHolder entriesMap(GridCacheContext cctx) { + return internal.entriesMap(cctx); + } + + /** {@inheritDoc} */ + @Nullable @Override protected CacheMapHolder entriesMapIfExists(Integer cacheId) { + return internal.entriesMapIfExists(cacheId); + } + + /** {@inheritDoc} */ + @Override public IgniteCacheOffheapManager.CacheDataStore dataStore() { + return internal.dataStore(); + } + + /** {@inheritDoc} */ + @Override public boolean addReservation(GridDhtPartitionsReservation r) { + return internal.addReservation(r); + } + + /** {@inheritDoc} */ + @Override public void removeReservation(GridDhtPartitionsReservation r) { + internal.removeReservation(r); + } + + /** {@inheritDoc} */ + @Override public int id() { + return internal.id(); + } + + /** {@inheritDoc} */ + @Override public GridDhtPartitionState state() { + return internal.state(); + } + + /** {@inheritDoc} */ + @Override public int reservations() { + return internal.reservations(); + } + + /** {@inheritDoc} */ + @Override public boolean isEmpty() { + return internal.isEmpty(); + } + + /** {@inheritDoc} */ + @Override public boolean valid() { + return internal.valid(); + } + + /** {@inheritDoc} */ + @Override public void cleanupRemoveQueue() { + internal.cleanupRemoveQueue(); + } + + /** {@inheritDoc} */ + @Override public void onDeferredDelete(int cacheId, KeyCacheObject key, GridCacheVersion ver) { + internal.onDeferredDelete(cacheId,key,ver); + } + + /** {@inheritDoc} */ + @Override public void lock() { + internal.lock(); + } + + /** {@inheritDoc} */ + @Override public void unlock() { + internal.unlock(); + } + + /** {@inheritDoc} */ + @Override public boolean reserve() { + return internal.reserve(); + } + + /** {@inheritDoc} */ + @Override public void release() { + internal.release(); + } + + /** {@inheritDoc} */ + @Override protected void release(int sizeChange, CacheMapHolder hld, GridCacheEntryEx e) { + internal.release(); + } + + /** {@inheritDoc} */ + @Override public void restoreState(GridDhtPartitionState stateToRestore) { + internal.restoreState(stateToRestore); + } + + /** {@inheritDoc} */ + @Override public void moving() { + + internal.moving(); + } + + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture rent(boolean updateSeq) { + return internal.rent(updateSeq); + } + + /** {@inheritDoc} */ + @Override public void clearAsync() { + internal.clearAsync(); + } + + /** {@inheritDoc} */ + @Override public boolean markForDestroy() { + return internal.markForDestroy(); + } + + /** {@inheritDoc} */ + @Override public void destroy() { + internal.destroy(); + } + + + /** {@inheritDoc} */ + @Override public void awaitDestroy() { + internal.awaitDestroy(); + } + + + /** {@inheritDoc} */ + @Override public void onClearFinished(IgniteInClosure> lsnr) { + internal.onClearFinished(lsnr); + } + + + /** {@inheritDoc} */ + @Override public boolean isClearing() { + return internal.isClearing(); + } + + + /** {@inheritDoc} */ + @Override public boolean tryClear(EvictionContext evictionCtx) throws NodeStoppingException { + return internal.tryClear(evictionCtx); + } + + /** {@inheritDoc} */ + @Override public boolean primary(AffinityTopologyVersion topVer) { + return internal.primary(topVer); + } + + + /** {@inheritDoc} */ + @Override public boolean backup(AffinityTopologyVersion topVer) { + return internal.backup(topVer); + } + + /** {@inheritDoc} */ + @Override public long initialUpdateCounter() { + return internal.initialUpdateCounter(); + } + + + /** {@inheritDoc} */ + @Override public void updateCounter(long val) { + + internal.updateCounter(val); + } + + + /** {@inheritDoc} */ + @Override public void initialUpdateCounter(long val) { + + internal.initialUpdateCounter(val); + } + + + /** {@inheritDoc} */ + @Override public long fullSize() { + + return internal.fullSize(); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + + return internal.hashCode(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + + return internal.equals(obj); + } + + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull GridDhtLocalPartition part) { + return internal.compareTo(part); + } + + /** {@inheritDoc} */ + @Override public String toString() { + + return internal.toString(); + } + + /** {@inheritDoc} */ + @Override public int publicSize(int cacheId) { + + return internal.publicSize(cacheId); + } + + /** {@inheritDoc} */ + @Override public void incrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { + internal.incrementPublicSize(hld,e); + } + + /** {@inheritDoc} */ + @Override public void decrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { + internal.decrementPublicSize(hld,e); + } + + /** {@inheritDoc} */ + @Nullable @Override public GridCacheMapEntry getEntry(GridCacheContext ctx, KeyCacheObject key) { + return internal.getEntry(ctx,key); + } + + /** {@inheritDoc} */ + @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent( + GridCacheContext ctx, + final AffinityTopologyVersion topVer, + KeyCacheObject key, + final boolean create, + final boolean touch) { + return internal.putEntryIfObsoleteOrAbsent(ctx, topVer, key, create, touch); + } + + /** {@inheritDoc} */ + @Override public boolean removeEntry(final GridCacheEntryEx entry) { + + return internal.removeEntry(entry); + } + + /** {@inheritDoc} */ + @Override public Collection entries(int cacheId, final CacheEntryPredicate... filter) { + return internal.entries(cacheId, filter); + } + + /** {@inheritDoc} */ + @Override public Set entrySet(int cacheId, final CacheEntryPredicate... filter) { + return internal.entrySet(cacheId, filter); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java new file mode 100644 index 0000000000000..570d568fee212 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java @@ -0,0 +1,560 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceArray; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; +import org.apache.ignite.internal.processors.cache.distributed.dht.MockGridDhtLocalPartition; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +import org.apache.ignite.internal.util.GridSpinBusyLock; +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.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; + +/** + * Test for 6 retry cases + */ +public class RetryCauseMessageTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int NODES_COUNT = 2; + + /** */ + private static final int ORG_COUNT = NODES_COUNT; + + /** */ + private static final int PERSON_PER_ORG_COUNT = 50; + /** */ + private static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + + "where Person.orgId = org.id " + + "and lower(org.name) = lower(?)"; + /** */ + private static final String ORG_SQL = "select * from Organization"; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + /** */ + private IgniteCache orgCache; + /** */ + private IgniteH2Indexing h2Idx; + + /** */ + @Override protected long getTestTimeout() { + return 600 * 1000; + } + + /** + * Failed to reserve partitions for query (cache is not found on local node) + */ + public void testCacheWasNotFoundMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + qryReq.caches().add(Integer.MAX_VALUE); + startedExecutor.onMessage(nodeId, msg); + qryReq.caches().remove(qryReq.caches().size() - 1); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (group reservation failed) + */ + public void testGrpReservationFailureMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + return false; + } + @Override public void release() {} + + }); + } + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (group reservation failed) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) + */ + public void testReplicatedCacheReserveFailureMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); + } + else + startedExecutor.onMessage(nodeId, msg); + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery<>(Organization.class, ORG_SQL); + qry.setDistributedJoins(true); + try { + orgCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) + */ + public void testPartitionedCacheReserveFailureMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) + */ + public void testPartitionStateChangedMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + return true; + } + + @Override public void release() { + + } + }); + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + + if (GridDhtPartitionTopologyImpl.class.isAssignableFrom(cctx.topology().getClass())) { + GridDhtPartitionTopologyImpl tpg = (GridDhtPartitionTopologyImpl)(cctx.topology()); + AtomicReferenceArray locParts = GridTestUtils.getFieldValue(tpg, GridDhtPartitionTopologyImpl.class, "locParts"); + GridDhtLocalPartition part = locParts.get(0); + MockGridDhtLocalPartition mockPart = new MockGridDhtLocalPartition( + GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "ctx"), + GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "grp"), + part) { + volatile private boolean preReserved = true; + + @Override public boolean reserve() { + preReserved = false; + return super.reserve(); + } + + @Override public GridDhtPartitionState state() { + if (preReserved) + return super.state(); + else + return MOVING; + } + }; + locParts.set(0, mockPart); + startedExecutor.onMessage(nodeId, msg); + locParts.set(0, part); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to execute non-collocated query (will retry) + */ + public void testNonCollocatedFailureMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + throw new GridH2RetryException("test retry exception"); + } + + @Override public void release() { + + } + }); + } + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to execute non-collocated query (will retry) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setPeerClassLoadingEnabled(false); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + disco.setForceServerMode(true); + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); + GridQueryProcessor qryProc = grid(ignite.name()).context().query(); + + h2Idx = GridTestUtils.getFieldValue(qryProc, GridQueryProcessor.class, "idx"); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setIndexedTypes(String.class, Person.class)); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, Organization.class) + ); + + awaitPartitionMapExchange(); + + populateDataIntoOrg(orgCache); + populateDataIntoPerson(personCache); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * Populate organization cache with test data + * @param cache @{IgniteCache} + */ + private void populateDataIntoOrg(IgniteCache cache) { + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + cache.put(org.getId(), org); + } + } + + /** + * Populate person cache with test data + * @param cache @{IgniteCache} + */ + private void populateDataIntoPerson(IgniteCache cache) { + int personId = 0; + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + + for (int j = 0; j < PERSON_PER_ORG_COUNT; j++) { + Person prsn = new Person(); + prsn.setId("pers" + personId); + prsn.setOrgId(org.getId()); + prsn.setName("Person name #" + personId); + + cache.put(prsn.getId(), prsn); + + personId++; + } + } + } + + /** + * + */ + private static class Person { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String orgId; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public String getId() { + return id; + } + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getOrgId() { + return orgId; + } + + /** */ + public void setOrgId(String orgId) { + this.orgId = orgId; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + + /** + * + */ + private static class Organization { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getId() { + return id; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + + /** + * Wrapper around @{GridMapQueryExecutor} + */ + private abstract static class MockGridMapQueryExecutor extends GridMapQueryExecutor { + + /** + * Wrapped executor + */ + GridMapQueryExecutor startedExecutor; + + /** */ + MockGridMapQueryExecutor insertRealExecutor(GridMapQueryExecutor realExecutor) { + this.startedExecutor = realExecutor; + return this; + } + + /** + * @param busyLock Busy lock. + */ + MockGridMapQueryExecutor(GridSpinBusyLock busyLock) { + super(busyLock); + } + + /** {@inheritDoc} */ + @Override public void onMessage(UUID nodeId, Object msg) { + + startedExecutor.onMessage(nodeId, msg); + } + + /** {@inheritDoc} */ + @Override public void cancelLazyWorkers() { + + startedExecutor.cancelLazyWorkers(); + } + + /** {@inheritDoc} */ + @Override GridSpinBusyLock busyLock() { + + return startedExecutor.busyLock(); + + } + + /** {@inheritDoc} */ + @Override public void onCacheStop(String cacheName) { + + startedExecutor.onCacheStop(cacheName); + } + + /** {@inheritDoc} */ + @Override public void stopAndUnregisterCurrentLazyWorker() { + startedExecutor.stopAndUnregisterCurrentLazyWorker(); + } + + /** {@inheritDoc} */ + @Override public void unregisterLazyWorker(MapQueryLazyWorker worker) { + startedExecutor.unregisterLazyWorker(worker); + } + + /** {@inheritDoc} */ + @Override public int registeredLazyWorkers() { + + return startedExecutor.registeredLazyWorkers(); + } + } + +} From 4587bfc24f25045fd7fc2197076797cc6ca54e32 Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 14 Aug 2018 16:23:19 +0300 Subject: [PATCH 02/95] Review. --- .../query/h2/twostep/GridMapQueryExecutor.java | 14 +++++++++++++- .../query/h2/twostep/GridReduceQueryExecutor.java | 3 +++ .../query/h2/twostep/ReduceQueryRun.java | 1 + 3 files changed, 17 insertions(+), 1 deletion(-) 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 1af66bbea5fb0..30ebe2fe1c63d 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 @@ -319,10 +319,15 @@ private String reservePartitions( // Cache was not found, probably was not deployed yet. if (cctx == null) { + // TODO: nodeId is ID of remote node, not current + + // TODO: CacheConfiguration.nodeFilter final String rslt = String.format("Failed to reserve partitions for query (cache is not found on local node) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", nodeId, reqId, topVer, cacheIds.get(i)); + logRetry(rslt); - return rslt; + + return rslt; // TODO: rslt -> res } if (cctx.isLocal() || !cctx.rebalanceEnabled()) @@ -336,6 +341,7 @@ private String reservePartitions( if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits. if (r != MapReplicatedReservation.INSTANCE) { if (!r.reserve()) { + // TODO: ??? - understand when it happens. final String rslt = String.format("Failed to reserve partitions for query (group reservation failed) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s]", nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); logRetry(rslt); @@ -357,6 +363,7 @@ private String reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING) { + // TODO: Play tiwh rebalance throttle/batch size, stop node final String rslt = String.format("Failed to reserve partitions for query " + "(partition of REPLICATED cache is not in OWNING state) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", @@ -388,6 +395,7 @@ private String reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING || !part.reserve()) { + // TODO: Stop/start nodes final String rslt = String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache cannot be reserved) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", @@ -410,6 +418,7 @@ private String reservePartitions( partState = part.state(); if (part.state() != OWNING) { + // TODO: Up to you, may skip tests. final String rslt = String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partState=%s]", @@ -693,6 +702,7 @@ private void onQueryRequest0( if (topVer != null) { // Reserve primary for topology version or explicit partitions. String reservationError = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); + if (!F.isEmpty(reservationError)) { // Unregister lazy worker because re-try may never reach this node again. if (lazy) @@ -814,7 +824,9 @@ private void onQueryRequest0( if (retryErr != null) { final String retryCause = String.format("Failed to execute non-collocated query (will retry) [nodeId=%s, reqId=%s, errMsg=%s]",node.id(),reqId,retryErr.getMessage()); + logRetry(retryCause); + sendRetry(node, reqId, segmentId,retryCause); } else { 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 fd9ca676da563..0a1ec28d54ff6 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 @@ -574,6 +574,7 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); + // TODO: Use previous query run final AtomicReference rootCause = new AtomicReference<>(); for (int attempt = 0;; attempt++) { @@ -849,8 +850,10 @@ public Iterator> query( if (retry) { assert r != null; + if (!F.isEmpty(r.rootCause())) rootCause.compareAndSet(null, r.rootCause()); + if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 0097048bab935..c84d258b9ef9d 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -58,6 +58,7 @@ class ReduceQueryRun { private final AtomicReference state = new AtomicReference<>(); /** Holder of root cause description*/ + // TODO: Single atmoic with (state, rootCause, nodeId) private final AtomicReference rootCause = new AtomicReference<>(); /** From 1511eb31b644113091be156d872f5adb2daecb84 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 14 Aug 2018 14:21:13 +0300 Subject: [PATCH 03/95] IGNITE-9141 Implemented --- .../messages/GridQueryNextPageResponse.java | 34 +- .../h2/twostep/GridMapQueryExecutor.java | 104 ++-- .../h2/twostep/GridReduceQueryExecutor.java | 20 +- .../query/h2/twostep/ReduceQueryRun.java | 21 + .../dht/MockGridDhtLocalPartition.java | 310 ++++++++++ .../h2/twostep/RetryCauseMessageTest.java | 560 ++++++++++++++++++ 6 files changed, 999 insertions(+), 50 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java index 4d918a04eacb8..925823c571c87 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java @@ -67,6 +67,9 @@ public class GridQueryNextPageResponse implements Message { /** */ private AffinityTopologyVersion retry; + /** Retry cause description*/ + private String retryCause; + /** Last page flag. */ private boolean last; @@ -230,6 +233,12 @@ public Collection plainRows() { return false; writer.incrementState(); + + case 9: + if (!writer.writeString("retryCause", retryCause)) + return false; + + writer.incrementState(); } return true; @@ -314,6 +323,15 @@ public Collection plainRows() { return false; reader.incrementState(); + + case 9: + retryCause = reader.readString("retryCause"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridQueryNextPageResponse.class); @@ -326,7 +344,7 @@ public Collection plainRows() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 9; + return 10; } /** @@ -343,6 +361,18 @@ public void retry(AffinityTopologyVersion retry) { this.retry = retry; } + /** + * @return Retry Ccause message. + */ + public String retryCause() { return retryCause; } + + /** + * @param retryCause Retry Ccause message. + */ + public void retryCause(String retryCause){ + this.retryCause = retryCause; + } + /** * @return Last page flag. */ @@ -363,4 +393,4 @@ public void last(boolean last) { "valsSize", vals != null ? vals.size() : 0, "rowsSize", plainRows != null ? plainRows.size() : 0); } -} \ No newline at end of file +} 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 216a259da73e4..1af66bbea5fb0 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 @@ -296,10 +296,10 @@ private GridDhtLocalPartition partition(GridCacheContext cctx, int p) { * @param reserved Reserved list. * @param nodeId Node ID. * @param reqId Request ID. - * @return {@code true} If all the needed partitions successfully reserved. + * @return String which is null in case of success or with causeMessage if failed * @throws IgniteCheckedException If failed. */ - private boolean reservePartitions( + private String reservePartitions( @Nullable List cacheIds, AffinityTopologyVersion topVer, final int[] explicitParts, @@ -310,7 +310,7 @@ private boolean reservePartitions( assert topVer != null; if (F.isEmpty(cacheIds)) - return true; + return null; Collection partIds = wrap(explicitParts); @@ -319,11 +319,10 @@ private boolean reservePartitions( // Cache was not found, probably was not deployed yet. if (cctx == null) { - logRetry("Failed to reserve partitions for query (cache is not found on local node) [" + - "rmtNodeId=" + nodeId + ", reqId=" + reqId + ", affTopVer=" + topVer + ", cacheId=" + - cacheIds.get(i) + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query (cache is not found on local node) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", nodeId, reqId, topVer, cacheIds.get(i)); + logRetry(rslt); + return rslt; } if (cctx.isLocal() || !cctx.rebalanceEnabled()) @@ -337,11 +336,10 @@ private boolean reservePartitions( if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits. if (r != MapReplicatedReservation.INSTANCE) { if (!r.reserve()) { - logRetry("Failed to reserve partitions for query (group reservation failed) [" + - "rmtNodeId=" + nodeId + ", reqId=" + reqId + ", affTopVer=" + topVer + - ", cacheId=" + cacheIds.get(i) + ", cacheName=" + cctx.name() + "]"); - - return false; // We need explicit partitions here -> retry. + final String rslt = String.format("Failed to reserve partitions for query (group reservation failed) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s]", nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); + logRetry(rslt); + return rslt; } reserved.add(r); @@ -359,13 +357,20 @@ private boolean reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING) { - logRetry("Failed to reserve partitions for query (partition of " + - "REPLICATED cache is not in OWNING state) [rmtNodeId=" + nodeId + - ", reqId=" + reqId + ", affTopVer=" + topVer + ", cacheId=" + cacheIds.get(i) + - ", cacheName=" + cctx.name() + ", part=" + p + ", partFound=" + (part != null) + - ", partState=" + partState + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query " + + "(partition of REPLICATED cache is not in OWNING state) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + p, + (part != null), + partState + ); + logRetry(rslt); + return rslt; } } @@ -383,13 +388,20 @@ private boolean reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING || !part.reserve()) { - logRetry("Failed to reserve partitions for query (partition of " + - "PARTITIONED cache cannot be reserved) [rmtNodeId=" + nodeId + ", reqId=" + reqId + - ", affTopVer=" + topVer + ", cacheId=" + cacheIds.get(i) + - ", cacheName=" + cctx.name() + ", part=" + partId + ", partFound=" + (part != null) + - ", partState=" + partState + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query " + + "(partition of PARTITIONED cache cannot be reserved) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + partId, + (part != null), + partState + ); + logRetry(rslt); + return rslt; } reserved.add(part); @@ -398,12 +410,19 @@ private boolean reservePartitions( partState = part.state(); if (part.state() != OWNING) { - logRetry("Failed to reserve partitions for query (partition of " + - "PARTITIONED cache is not in OWNING state after reservation) [rmtNodeId=" + nodeId + - ", reqId=" + reqId + ", affTopVer=" + topVer + ", cacheId=" + cacheIds.get(i) + - ", cacheName=" + cctx.name() + ", part=" + partId + ", partState=" + partState + "]"); - - return false; + final String rslt = String.format("Failed to reserve partitions for query " + + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partState=%s]", + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + partId, + partState + ); + logRetry(rslt); + return rslt; } } @@ -426,7 +445,7 @@ private boolean reservePartitions( } } - return true; + return null; } /** @@ -673,12 +692,13 @@ private void onQueryRequest0( try { if (topVer != null) { // Reserve primary for topology version or explicit partitions. - if (!reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId)) { + String reservationError = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); + if (!F.isEmpty(reservationError)) { // Unregister lazy worker because re-try may never reach this node again. if (lazy) stopAndUnregisterCurrentLazyWorker(); - sendRetry(node, reqId, segmentId); + sendRetry(node, reqId, segmentId, reservationError); return; } @@ -793,10 +813,9 @@ private void onQueryRequest0( GridH2RetryException retryErr = X.cause(e, GridH2RetryException.class); if (retryErr != null) { - logRetry("Failed to execute non-collocated query (will retry) [nodeId=" + node.id() + - ", reqId=" + reqId + ", errMsg=" + retryErr.getMessage() + ']'); - - sendRetry(node, reqId, segmentId); + final String retryCause = String.format("Failed to execute non-collocated query (will retry) [nodeId=%s, reqId=%s, errMsg=%s]",node.id(),reqId,retryErr.getMessage()); + logRetry(retryCause); + sendRetry(node, reqId, segmentId,retryCause); } else { U.error(log, "Failed to execute local query.", e); @@ -845,7 +864,7 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th List reserved = new ArrayList<>(); - if (!reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId)) { + if (!F.isEmpty(reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId))) { U.error(log, "Failed to reserve partitions for DML request. [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", reqId=" + req.requestId() + ", cacheIds=" + cacheIds + ", topVer=" + topVer + ", parts=" + Arrays.toString(parts) + ']'); @@ -1081,7 +1100,7 @@ private void sendNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryRes * @param reqId Request ID. * @param segmentId Index segment ID. */ - private void sendRetry(ClusterNode node, long reqId, int segmentId) { + private void sendRetry(ClusterNode node, long reqId, int segmentId, String retryCause) { try { boolean loc = node.isLocal(); @@ -1092,6 +1111,7 @@ private void sendRetry(ClusterNode node, long reqId, int segmentId) { false); msg.retry(h2.readyTopologyVersion()); + msg.retryCause(retryCause); if (loc) h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg); 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 65aeae927cdf0..fd9ca676da563 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 @@ -38,6 +38,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import javax.cache.CacheException; @@ -289,7 +290,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { * @param node Node. * @param msg Message. */ - private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { + private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse msg) { final long qryReqId = msg.queryRequestId(); final int qry = msg.query(); final int seg = msg.segmentId(); @@ -316,7 +317,8 @@ private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { if (err0 != null && err0.getCause() instanceof IgniteClientDisconnectedException) throw err0; - CacheException e = new CacheException("Failed to fetch data from node: " + node.id()); + CacheException e = new CacheException( + (msg.retryCause()!=null) ? msg.retryCause() : "Failed to fetch data from node: " + node.id()); if (err0 != null) e.addSuppressed(err0); @@ -349,7 +351,7 @@ private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { idx.addPage(page); if (msg.retry() != null) - retry(r, msg.retry(), node.id()); + r.stateWithMsg(msg, node.id()); else if (msg.page() == 0) // Do count down on each first page received. r.latch().countDown(); } @@ -572,10 +574,13 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); - for (int attempt = 0;; attempt++) { - if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) - throw new CacheException("Failed to map SQL query to topology."); + final AtomicReference rootCause = new AtomicReference<>(); + for (int attempt = 0;; attempt++) { + if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) { + String rcValue = rootCause.get(); + throw new CacheException((!F.isEmpty(rcValue))?rcValue:("Failed to map SQL query to topology.")); + } if (attempt != 0) { try { Thread.sleep(attempt * 10); // Wait for exchange. @@ -843,6 +848,9 @@ public Iterator> query( } if (retry) { + assert r != null; + if (!F.isEmpty(r.rootCause())) + rootCause.compareAndSet(null, r.rootCause()); if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 73bb002ee1635..0097048bab935 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse; import org.h2.jdbc.JdbcConnection; import org.jetbrains.annotations.Nullable; @@ -56,6 +57,9 @@ class ReduceQueryRun { /** Can be either CacheException in case of error or AffinityTopologyVersion to retry if needed. */ private final AtomicReference state = new AtomicReference<>(); + /** Holder of root cause description*/ + private final AtomicReference rootCause = new AtomicReference<>(); + /** * Constructor. * @@ -99,6 +103,16 @@ void state(Object o, @Nullable UUID nodeId) { idx.fail(nodeId, e); } + /** + * @param msg corresponding response message + * @param nodeId Node ID. + */ + void stateWithMsg(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { + assert msg != null; + rootCause.compareAndSet(null, msg.retryCause()); + state(msg.retry(), nodeId); + } + /** * @param e Error. */ @@ -134,6 +148,13 @@ Object state() { return state.get(); } + /** + * @return Root Cause. + */ + String rootCause() { + return rootCause.get(); + } + /** * @return Indexes. */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java new file mode 100644 index 0000000000000..33d7613c9c99a --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java @@ -0,0 +1,310 @@ +package org.apache.ignite.internal.processors.cache.distributed.dht; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteInClosure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Wrapper around GridDhtLocalPartition to be extended in test cases + */ +public abstract class MockGridDhtLocalPartition extends GridDhtLocalPartition{ + /** + * fake id generator in order to bypass construction stage failure (if invoked with real id) + */ + private static AtomicInteger cntr = new AtomicInteger(Integer.MAX_VALUE); + + /** + * Real object + */ + private GridDhtLocalPartition internal; + + /** + * @param ctx Context. + * @param grp Cache group. + * @param id Partition ID. + */ + private MockGridDhtLocalPartition(GridCacheSharedContext ctx, + CacheGroupContext grp, int id) { + super(ctx, grp, id); + } + + /** */ + protected MockGridDhtLocalPartition(GridCacheSharedContext ctx, + CacheGroupContext grp, GridDhtLocalPartition internal){ + this(ctx, grp, cntr.getAndDecrement()); + this.internal = internal; + } + + /** */ + protected GridDhtLocalPartition getInternal(){ + return internal; + } + + /** {@inheritDoc} */ + @Override public int internalSize() { + return internal.internalSize(); + } + + /** {@inheritDoc} */ + @Override protected CacheMapHolder entriesMap(GridCacheContext cctx) { + return internal.entriesMap(cctx); + } + + /** {@inheritDoc} */ + @Nullable @Override protected CacheMapHolder entriesMapIfExists(Integer cacheId) { + return internal.entriesMapIfExists(cacheId); + } + + /** {@inheritDoc} */ + @Override public IgniteCacheOffheapManager.CacheDataStore dataStore() { + return internal.dataStore(); + } + + /** {@inheritDoc} */ + @Override public boolean addReservation(GridDhtPartitionsReservation r) { + return internal.addReservation(r); + } + + /** {@inheritDoc} */ + @Override public void removeReservation(GridDhtPartitionsReservation r) { + internal.removeReservation(r); + } + + /** {@inheritDoc} */ + @Override public int id() { + return internal.id(); + } + + /** {@inheritDoc} */ + @Override public GridDhtPartitionState state() { + return internal.state(); + } + + /** {@inheritDoc} */ + @Override public int reservations() { + return internal.reservations(); + } + + /** {@inheritDoc} */ + @Override public boolean isEmpty() { + return internal.isEmpty(); + } + + /** {@inheritDoc} */ + @Override public boolean valid() { + return internal.valid(); + } + + /** {@inheritDoc} */ + @Override public void cleanupRemoveQueue() { + internal.cleanupRemoveQueue(); + } + + /** {@inheritDoc} */ + @Override public void onDeferredDelete(int cacheId, KeyCacheObject key, GridCacheVersion ver) { + internal.onDeferredDelete(cacheId,key,ver); + } + + /** {@inheritDoc} */ + @Override public void lock() { + internal.lock(); + } + + /** {@inheritDoc} */ + @Override public void unlock() { + internal.unlock(); + } + + /** {@inheritDoc} */ + @Override public boolean reserve() { + return internal.reserve(); + } + + /** {@inheritDoc} */ + @Override public void release() { + internal.release(); + } + + /** {@inheritDoc} */ + @Override protected void release(int sizeChange, CacheMapHolder hld, GridCacheEntryEx e) { + internal.release(); + } + + /** {@inheritDoc} */ + @Override public void restoreState(GridDhtPartitionState stateToRestore) { + internal.restoreState(stateToRestore); + } + + /** {@inheritDoc} */ + @Override public void moving() { + + internal.moving(); + } + + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture rent(boolean updateSeq) { + return internal.rent(updateSeq); + } + + /** {@inheritDoc} */ + @Override public void clearAsync() { + internal.clearAsync(); + } + + /** {@inheritDoc} */ + @Override public boolean markForDestroy() { + return internal.markForDestroy(); + } + + /** {@inheritDoc} */ + @Override public void destroy() { + internal.destroy(); + } + + + /** {@inheritDoc} */ + @Override public void awaitDestroy() { + internal.awaitDestroy(); + } + + + /** {@inheritDoc} */ + @Override public void onClearFinished(IgniteInClosure> lsnr) { + internal.onClearFinished(lsnr); + } + + + /** {@inheritDoc} */ + @Override public boolean isClearing() { + return internal.isClearing(); + } + + + /** {@inheritDoc} */ + @Override public boolean tryClear(EvictionContext evictionCtx) throws NodeStoppingException { + return internal.tryClear(evictionCtx); + } + + /** {@inheritDoc} */ + @Override public boolean primary(AffinityTopologyVersion topVer) { + return internal.primary(topVer); + } + + + /** {@inheritDoc} */ + @Override public boolean backup(AffinityTopologyVersion topVer) { + return internal.backup(topVer); + } + + /** {@inheritDoc} */ + @Override public long initialUpdateCounter() { + return internal.initialUpdateCounter(); + } + + + /** {@inheritDoc} */ + @Override public void updateCounter(long val) { + + internal.updateCounter(val); + } + + + /** {@inheritDoc} */ + @Override public void initialUpdateCounter(long val) { + + internal.initialUpdateCounter(val); + } + + + /** {@inheritDoc} */ + @Override public long fullSize() { + + return internal.fullSize(); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + + return internal.hashCode(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + + return internal.equals(obj); + } + + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull GridDhtLocalPartition part) { + return internal.compareTo(part); + } + + /** {@inheritDoc} */ + @Override public String toString() { + + return internal.toString(); + } + + /** {@inheritDoc} */ + @Override public int publicSize(int cacheId) { + + return internal.publicSize(cacheId); + } + + /** {@inheritDoc} */ + @Override public void incrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { + internal.incrementPublicSize(hld,e); + } + + /** {@inheritDoc} */ + @Override public void decrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { + internal.decrementPublicSize(hld,e); + } + + /** {@inheritDoc} */ + @Nullable @Override public GridCacheMapEntry getEntry(GridCacheContext ctx, KeyCacheObject key) { + return internal.getEntry(ctx,key); + } + + /** {@inheritDoc} */ + @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent( + GridCacheContext ctx, + final AffinityTopologyVersion topVer, + KeyCacheObject key, + final boolean create, + final boolean touch) { + return internal.putEntryIfObsoleteOrAbsent(ctx, topVer, key, create, touch); + } + + /** {@inheritDoc} */ + @Override public boolean removeEntry(final GridCacheEntryEx entry) { + + return internal.removeEntry(entry); + } + + /** {@inheritDoc} */ + @Override public Collection entries(int cacheId, final CacheEntryPredicate... filter) { + return internal.entries(cacheId, filter); + } + + /** {@inheritDoc} */ + @Override public Set entrySet(int cacheId, final CacheEntryPredicate... filter) { + return internal.entrySet(cacheId, filter); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java new file mode 100644 index 0000000000000..570d568fee212 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java @@ -0,0 +1,560 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceArray; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; +import org.apache.ignite.internal.processors.cache.distributed.dht.MockGridDhtLocalPartition; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +import org.apache.ignite.internal.util.GridSpinBusyLock; +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.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; + +/** + * Test for 6 retry cases + */ +public class RetryCauseMessageTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int NODES_COUNT = 2; + + /** */ + private static final int ORG_COUNT = NODES_COUNT; + + /** */ + private static final int PERSON_PER_ORG_COUNT = 50; + /** */ + private static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + + "where Person.orgId = org.id " + + "and lower(org.name) = lower(?)"; + /** */ + private static final String ORG_SQL = "select * from Organization"; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + /** */ + private IgniteCache orgCache; + /** */ + private IgniteH2Indexing h2Idx; + + /** */ + @Override protected long getTestTimeout() { + return 600 * 1000; + } + + /** + * Failed to reserve partitions for query (cache is not found on local node) + */ + public void testCacheWasNotFoundMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + qryReq.caches().add(Integer.MAX_VALUE); + startedExecutor.onMessage(nodeId, msg); + qryReq.caches().remove(qryReq.caches().size() - 1); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (group reservation failed) + */ + public void testGrpReservationFailureMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + return false; + } + @Override public void release() {} + + }); + } + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (group reservation failed) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) + */ + public void testReplicatedCacheReserveFailureMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); + } + else + startedExecutor.onMessage(nodeId, msg); + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery<>(Organization.class, ORG_SQL); + qry.setDistributedJoins(true); + try { + orgCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) + */ + public void testPartitionedCacheReserveFailureMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) + */ + public void testPartitionStateChangedMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + return true; + } + + @Override public void release() { + + } + }); + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + + if (GridDhtPartitionTopologyImpl.class.isAssignableFrom(cctx.topology().getClass())) { + GridDhtPartitionTopologyImpl tpg = (GridDhtPartitionTopologyImpl)(cctx.topology()); + AtomicReferenceArray locParts = GridTestUtils.getFieldValue(tpg, GridDhtPartitionTopologyImpl.class, "locParts"); + GridDhtLocalPartition part = locParts.get(0); + MockGridDhtLocalPartition mockPart = new MockGridDhtLocalPartition( + GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "ctx"), + GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "grp"), + part) { + volatile private boolean preReserved = true; + + @Override public boolean reserve() { + preReserved = false; + return super.reserve(); + } + + @Override public GridDhtPartitionState state() { + if (preReserved) + return super.state(); + else + return MOVING; + } + }; + locParts.set(0, mockPart); + startedExecutor.onMessage(nodeId, msg); + locParts.set(0, part); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to execute non-collocated query (will retry) + */ + public void testNonCollocatedFailureMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + throw new GridH2RetryException("test retry exception"); + } + + @Override public void release() { + + } + }); + } + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to execute non-collocated query (will retry) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setPeerClassLoadingEnabled(false); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + disco.setForceServerMode(true); + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); + GridQueryProcessor qryProc = grid(ignite.name()).context().query(); + + h2Idx = GridTestUtils.getFieldValue(qryProc, GridQueryProcessor.class, "idx"); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setIndexedTypes(String.class, Person.class)); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, Organization.class) + ); + + awaitPartitionMapExchange(); + + populateDataIntoOrg(orgCache); + populateDataIntoPerson(personCache); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * Populate organization cache with test data + * @param cache @{IgniteCache} + */ + private void populateDataIntoOrg(IgniteCache cache) { + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + cache.put(org.getId(), org); + } + } + + /** + * Populate person cache with test data + * @param cache @{IgniteCache} + */ + private void populateDataIntoPerson(IgniteCache cache) { + int personId = 0; + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + + for (int j = 0; j < PERSON_PER_ORG_COUNT; j++) { + Person prsn = new Person(); + prsn.setId("pers" + personId); + prsn.setOrgId(org.getId()); + prsn.setName("Person name #" + personId); + + cache.put(prsn.getId(), prsn); + + personId++; + } + } + } + + /** + * + */ + private static class Person { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String orgId; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public String getId() { + return id; + } + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getOrgId() { + return orgId; + } + + /** */ + public void setOrgId(String orgId) { + this.orgId = orgId; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + + /** + * + */ + private static class Organization { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getId() { + return id; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + + /** + * Wrapper around @{GridMapQueryExecutor} + */ + private abstract static class MockGridMapQueryExecutor extends GridMapQueryExecutor { + + /** + * Wrapped executor + */ + GridMapQueryExecutor startedExecutor; + + /** */ + MockGridMapQueryExecutor insertRealExecutor(GridMapQueryExecutor realExecutor) { + this.startedExecutor = realExecutor; + return this; + } + + /** + * @param busyLock Busy lock. + */ + MockGridMapQueryExecutor(GridSpinBusyLock busyLock) { + super(busyLock); + } + + /** {@inheritDoc} */ + @Override public void onMessage(UUID nodeId, Object msg) { + + startedExecutor.onMessage(nodeId, msg); + } + + /** {@inheritDoc} */ + @Override public void cancelLazyWorkers() { + + startedExecutor.cancelLazyWorkers(); + } + + /** {@inheritDoc} */ + @Override GridSpinBusyLock busyLock() { + + return startedExecutor.busyLock(); + + } + + /** {@inheritDoc} */ + @Override public void onCacheStop(String cacheName) { + + startedExecutor.onCacheStop(cacheName); + } + + /** {@inheritDoc} */ + @Override public void stopAndUnregisterCurrentLazyWorker() { + startedExecutor.stopAndUnregisterCurrentLazyWorker(); + } + + /** {@inheritDoc} */ + @Override public void unregisterLazyWorker(MapQueryLazyWorker worker) { + startedExecutor.unregisterLazyWorker(worker); + } + + /** {@inheritDoc} */ + @Override public int registeredLazyWorkers() { + + return startedExecutor.registeredLazyWorkers(); + } + } + +} From 0425a32a3d490165b29bcf9236a49a48f7761666 Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 14 Aug 2018 16:23:19 +0300 Subject: [PATCH 04/95] Review. --- .../query/h2/twostep/GridMapQueryExecutor.java | 14 +++++++++++++- .../query/h2/twostep/GridReduceQueryExecutor.java | 3 +++ .../query/h2/twostep/ReduceQueryRun.java | 1 + 3 files changed, 17 insertions(+), 1 deletion(-) 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 1af66bbea5fb0..30ebe2fe1c63d 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 @@ -319,10 +319,15 @@ private String reservePartitions( // Cache was not found, probably was not deployed yet. if (cctx == null) { + // TODO: nodeId is ID of remote node, not current + + // TODO: CacheConfiguration.nodeFilter final String rslt = String.format("Failed to reserve partitions for query (cache is not found on local node) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", nodeId, reqId, topVer, cacheIds.get(i)); + logRetry(rslt); - return rslt; + + return rslt; // TODO: rslt -> res } if (cctx.isLocal() || !cctx.rebalanceEnabled()) @@ -336,6 +341,7 @@ private String reservePartitions( if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits. if (r != MapReplicatedReservation.INSTANCE) { if (!r.reserve()) { + // TODO: ??? - understand when it happens. final String rslt = String.format("Failed to reserve partitions for query (group reservation failed) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s]", nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); logRetry(rslt); @@ -357,6 +363,7 @@ private String reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING) { + // TODO: Play tiwh rebalance throttle/batch size, stop node final String rslt = String.format("Failed to reserve partitions for query " + "(partition of REPLICATED cache is not in OWNING state) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", @@ -388,6 +395,7 @@ private String reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING || !part.reserve()) { + // TODO: Stop/start nodes final String rslt = String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache cannot be reserved) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", @@ -410,6 +418,7 @@ private String reservePartitions( partState = part.state(); if (part.state() != OWNING) { + // TODO: Up to you, may skip tests. final String rslt = String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partState=%s]", @@ -693,6 +702,7 @@ private void onQueryRequest0( if (topVer != null) { // Reserve primary for topology version or explicit partitions. String reservationError = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); + if (!F.isEmpty(reservationError)) { // Unregister lazy worker because re-try may never reach this node again. if (lazy) @@ -814,7 +824,9 @@ private void onQueryRequest0( if (retryErr != null) { final String retryCause = String.format("Failed to execute non-collocated query (will retry) [nodeId=%s, reqId=%s, errMsg=%s]",node.id(),reqId,retryErr.getMessage()); + logRetry(retryCause); + sendRetry(node, reqId, segmentId,retryCause); } else { 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 fd9ca676da563..0a1ec28d54ff6 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 @@ -574,6 +574,7 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); + // TODO: Use previous query run final AtomicReference rootCause = new AtomicReference<>(); for (int attempt = 0;; attempt++) { @@ -849,8 +850,10 @@ public Iterator> query( if (retry) { assert r != null; + if (!F.isEmpty(r.rootCause())) rootCause.compareAndSet(null, r.rootCause()); + if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 0097048bab935..c84d258b9ef9d 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -58,6 +58,7 @@ class ReduceQueryRun { private final AtomicReference state = new AtomicReference<>(); /** Holder of root cause description*/ + // TODO: Single atmoic with (state, rootCause, nodeId) private final AtomicReference rootCause = new AtomicReference<>(); /** From a7209f870148f984cdbf3e6437337c7b665defb5 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Mon, 20 Aug 2018 14:56:11 +0300 Subject: [PATCH 05/95] IGNITE-9141 Modified according to review comments. Integration tests added --- .../h2/twostep/GridMapQueryExecutor.java | 58 +++--- .../h2/twostep/GridReduceQueryExecutor.java | 33 ++-- .../query/h2/twostep/ReduceQueryRun.java | 135 +++++++++++--- ...DisappearedCacheCauseRetryMessageTest.java | 106 +++++++++++ ...isappearedCacheWasNotFoundMessageTest.java | 96 ++++++++++ .../query/h2/twostep/JoinSqlTestHelper.java | 138 ++++++++++++++ .../NonCollocatedRetryMessageTest.java | 112 ++++++++++++ .../h2/twostep/RetryCauseMessageTest.java | 168 +++--------------- 8 files changed, 630 insertions(+), 216 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java 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 30ebe2fe1c63d..c7f2d916d3f64 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 @@ -319,15 +319,12 @@ private String reservePartitions( // Cache was not found, probably was not deployed yet. if (cctx == null) { - // TODO: nodeId is ID of remote node, not current + final String res = String.format("Failed to reserve partitions for query (cache is not found on local node) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i)); - // TODO: CacheConfiguration.nodeFilter - final String rslt = String.format("Failed to reserve partitions for query (cache is not found on local node) [" + - "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", nodeId, reqId, topVer, cacheIds.get(i)); + logRetry(res); - logRetry(rslt); - - return rslt; // TODO: rslt -> res + return res; } if (cctx.isLocal() || !cctx.rebalanceEnabled()) @@ -341,11 +338,10 @@ private String reservePartitions( if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits. if (r != MapReplicatedReservation.INSTANCE) { if (!r.reserve()) { - // TODO: ??? - understand when it happens. - final String rslt = String.format("Failed to reserve partitions for query (group reservation failed) [" + - "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s]", nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); - logRetry(rslt); - return rslt; + final String res = String.format("Failed to reserve partitions for query (group reservation failed) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s]",ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); + logRetry(res); + return res; } reserved.add(r); @@ -363,10 +359,11 @@ private String reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING) { - // TODO: Play tiwh rebalance throttle/batch size, stop node - final String rslt = String.format("Failed to reserve partitions for query " + + + final String res = String.format("Failed to reserve partitions for query " + "(partition of REPLICATED cache is not in OWNING state) [" + - "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + ctx.localNodeId(), nodeId, reqId, topVer, @@ -376,8 +373,8 @@ private String reservePartitions( (part != null), partState ); - logRetry(rslt); - return rslt; + logRetry(res); + return res; } } @@ -395,10 +392,11 @@ private String reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; if (partState != OWNING || !part.reserve()) { - // TODO: Stop/start nodes - final String rslt = String.format("Failed to reserve partitions for query " + + + final String res = String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache cannot be reserved) [" + - "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + ctx.localNodeId(), nodeId, reqId, topVer, @@ -408,8 +406,8 @@ private String reservePartitions( (part != null), partState ); - logRetry(rslt); - return rslt; + logRetry(res); + return res; } reserved.add(part); @@ -418,10 +416,11 @@ private String reservePartitions( partState = part.state(); if (part.state() != OWNING) { - // TODO: Up to you, may skip tests. - final String rslt = String.format("Failed to reserve partitions for query " + + + final String res = String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + - "rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partState=%s]", + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partState=%s]", + ctx.localNodeId(), nodeId, reqId, topVer, @@ -430,8 +429,8 @@ private String reservePartitions( partId, partState ); - logRetry(rslt); - return rslt; + logRetry(res); + return res; } } @@ -823,7 +822,10 @@ private void onQueryRequest0( GridH2RetryException retryErr = X.cause(e, GridH2RetryException.class); if (retryErr != null) { - final String retryCause = String.format("Failed to execute non-collocated query (will retry) [nodeId=%s, reqId=%s, errMsg=%s]",node.id(),reqId,retryErr.getMessage()); + final String retryCause = String.format( + "Failed to execute non-collocated query (will retry) [localNodeId=%s, rmtNodeId=%s, reqId=%s, errMsg=%s]", + ctx.localNodeId(), node.id(), reqId, retryErr.getMessage() + ); logRetry(retryCause); 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 0a1ec28d54ff6..47b9588ffc2af 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 @@ -309,10 +309,9 @@ private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse try { page = new GridResultPage(ctx, node.id(), msg) { @Override public void fetchNextPage() { - Object errState = r.state(); - if (errState != null) { - CacheException err0 = errState instanceof CacheException ? (CacheException)errState : null; + if (r.hasError()) { + CacheException err0 = r.cacheEx(); if (err0 != null && err0.getCause() instanceof IgniteClientDisconnectedException) throw err0; @@ -371,6 +370,10 @@ private void retry(ReduceQueryRun r, AffinityTopologyVersion retryVer, UUID node */ private boolean isPreloadingActive(List cacheIds) { for (Integer cacheId : cacheIds) { + + if (null == cacheContext(cacheId)) + throw new CacheException(String.format("Grid cache context is not registered for cache id=%s",cacheId)); + if (hasMovingPartitions(cacheContext(cacheId))) return true; } @@ -383,6 +386,7 @@ private boolean isPreloadingActive(List cacheIds) { * @return {@code True} If cache has partitions in {@link GridDhtPartitionState#MOVING} state. */ private boolean hasMovingPartitions(GridCacheContext cctx) { + assert cctx!=null; return !cctx.isLocal() && cctx.topology().hasMovingPartitions(); } @@ -574,12 +578,11 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); - // TODO: Use previous query run - final AtomicReference rootCause = new AtomicReference<>(); + ReduceQueryRun prevR = null; for (int attempt = 0;; attempt++) { if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) { - String rcValue = rootCause.get(); + String rcValue = prevR.rootCause(); throw new CacheException((!F.isEmpty(rcValue))?rcValue:("Failed to map SQL query to topology.")); } if (attempt != 0) { @@ -783,11 +786,9 @@ public Iterator> query( if (send(nodes, req, parts == null ? null : new ExplicitPartitionsSpecializer(qryMap), false)) { awaitAllReplies(r, nodes, cancel); - Object state = r.state(); - - if (state != null) { - if (state instanceof CacheException) { - CacheException err = (CacheException)state; + if (r.hasError()) { + if (r.cacheEx() != null) { + CacheException err = r.cacheEx(); if (err.getCause() instanceof IgniteClientDisconnectedException) throw err; @@ -796,13 +797,11 @@ public Iterator> query( throw new QueryCancelledException(); // Throw correct exception. throw new CacheException("Failed to run map query remotely." + err.getMessage(), err); - } - - if (state instanceof AffinityTopologyVersion) { + } else { retry = true; // If remote node asks us to retry then we have outdated full partition map. - h2.awaitForReadyTopologyVersion((AffinityTopologyVersion)state); + h2.awaitForReadyTopologyVersion(r.atv()); } } } @@ -850,9 +849,7 @@ public Iterator> query( if (retry) { assert r != null; - - if (!F.isEmpty(r.rootCause())) - rootCause.compareAndSet(null, r.rootCause()); + prevR=r; if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index c84d258b9ef9d..2f37370cda7a9 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -54,12 +54,9 @@ class ReduceQueryRun { /** */ private final int pageSize; - /** Can be either CacheException in case of error or AffinityTopologyVersion to retry if needed. */ - private final AtomicReference state = new AtomicReference<>(); + /** */ + private final AtomicReference state = new AtomicReference<>(); - /** Holder of root cause description*/ - // TODO: Single atmoic with (state, rootCause, nodeId) - private final AtomicReference rootCause = new AtomicReference<>(); /** * Constructor. @@ -91,17 +88,11 @@ class ReduceQueryRun { void state(Object o, @Nullable UUID nodeId) { assert o != null; assert o instanceof CacheException || o instanceof AffinityTopologyVersion : o.getClass(); - - if (!state.compareAndSet(null, o)) - return; - - while (latch.getCount() != 0) // We don't need to wait for all nodes to reply. - latch.countDown(); - - CacheException e = o instanceof CacheException ? (CacheException) o : null; - - for (GridMergeIndex idx : idxs) // Fail all merge indexes. - idx.fail(nodeId, e); + StateBuilder sb = State.getStateBuilder().nodeId(nodeId); + if ( o instanceof CacheException ) + sb.exception((CacheException)o); + else sb.atv( (AffinityTopologyVersion)o); + state(sb.build()); } /** @@ -110,8 +101,19 @@ void state(Object o, @Nullable UUID nodeId) { */ void stateWithMsg(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { assert msg != null; - rootCause.compareAndSet(null, msg.retryCause()); - state(msg.retry(), nodeId); + assert msg.retry() != null; + state(State.getStateBuilder().atv(msg.retry()).rootCause(msg.retryCause()).nodeId(nodeId).build()); + } + + private void state(State state){ + if (!this.state.compareAndSet(null, state)) + return; + + while (latch.getCount() != 0) // We don't need to wait for all nodes to reply. + latch.countDown(); + + for (GridMergeIndex idx : idxs) // Fail all merge indexes. + idx.fail(state.nodeId, state.ex); } /** @@ -142,20 +144,24 @@ JdbcConnection connection() { return conn; } - /** - * @return State. - */ - Object state() { - return state.get(); + boolean hasError(){ + return state.get()!=null; } - /** - * @return Root Cause. - */ - String rootCause() { - return rootCause.get(); + CacheException cacheEx() { + State st = state.get(); + return st!=null ? st.ex : null; + } + + AffinityTopologyVersion atv(){ + State st = state.get(); + return st!=null ? st.atv : null; } + String rootCause(){ + State st = state.get(); + return st!=null ? st.rootCause : null; + } /** * @return Indexes. */ @@ -176,4 +182,77 @@ CountDownLatch latch() { void latch(CountDownLatch latch) { this.latch = latch; } + + /** */ + private static class State{ + + /** */ + private static StateBuilder getStateBuilder(){ + return new StateBuilder(); + } + /** */ + private final CacheException ex; + + /** */ + private final String rootCause; + + /** */ + private final AffinityTopologyVersion atv; + + /** */ + private final UUID nodeId; + + /** */ + private State(CacheException ex, String rootCause, AffinityTopologyVersion atv, UUID nodeId){ + this.ex=ex; + this.rootCause = rootCause; + this.atv = atv; + this.nodeId = nodeId; + } + } + + /** */ + private static class StateBuilder{ + /** */ + private CacheException ex = null; + + /** */ + private String rootCause = null; + + /** */ + private AffinityTopologyVersion atv = null; + + /** */ + private UUID nodeId = null; + + /** */ + private State build(){ + return new State(ex, rootCause, atv, nodeId); + } + + /** */ + private StateBuilder exception(CacheException ex){ + this.ex = ex; + return this; + } + + /** */ + private StateBuilder rootCause(String rootCause){ + this.rootCause=rootCause; + return this; + } + + /** */ + private StateBuilder atv(AffinityTopologyVersion atv){ + this.atv=atv; + return this; + } + + /** */ + private StateBuilder nodeId(UUID nodeId){ + this.nodeId = nodeId; + return this; + } + + } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java new file mode 100644 index 0000000000000..0072fe4a0a79a --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java @@ -0,0 +1,106 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import javax.cache.CacheException; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheMode; +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.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +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.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; + +/** + * Failed to reserve partitions for query (cache is not found on local node) Root cause test + */ +public class DisappearedCacheCauseRetryMessageTest extends GridCommonAbstractTest { + + /** */ + private static final int NODES_COUNT = 2; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + /** */ + private IgniteCache orgCache; + + public void testDisappearedCacheCauseRetryMessage() { + + SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + fail("No CacheException emitted."); + } + catch (CacheException e) { + assertTrue(e.getMessage(), e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + } + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + + volatile long reqId = -1; + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ + GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + reqId = req.requestId(); + orgCache.destroy(); + } else if ( GridQueryCancelRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridQueryCancelRequest req = (GridQueryCancelRequest) (gridMsg.message()); + + if (reqId == req.queryRequestId()) + orgCache = DisappearedCacheCauseRetryMessageTest.this.ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) + ); + + } + } + super.sendMessage(node, msg, ackC); + } + }); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + startGridsMultiThreaded(NODES_COUNT, false); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) + ); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) + ); + + awaitPartitionMapExchange(); + + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); + } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java new file mode 100644 index 0000000000000..ce106a8136877 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java @@ -0,0 +1,96 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheMode; +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.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +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.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; + +/** + * Grid cache context is not registered for cache id root cause message test + */ +public class DisappearedCacheWasNotFoundMessageTest extends GridCommonAbstractTest { + + /** */ + private static final int NODES_COUNT = 2; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + /** */ + private IgniteCache orgCache; + + public void testDisappearedCacheWasNotFoundMessage() { + + SqlQuery qry = new SqlQuery(Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + fail("No CacheException emitted."); + } + catch (CacheException e) { + assertTrue(e.getMessage(), e.getMessage().contains("Grid cache context is not registered for cache id")); + } + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ + GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + req.requestId(); + orgCache.destroy(); + } + } + super.sendMessage(node, msg, ackC); + } + }); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) + ); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, Organization.class) + ); + + awaitPartitionMapExchange(); + + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java new file mode 100644 index 0000000000000..f9c3c481487e9 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java @@ -0,0 +1,138 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.annotations.QuerySqlField; + +/** + * Join sql test helper + */ +public class JoinSqlTestHelper { + + /** */ + private static final int ORG_COUNT = 100; + + /** */ + private static final int PERSON_PER_ORG_COUNT = 10; + /** */ + public static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + + "where Person.orgId = org.id " + + "and lower(org.name) = lower(?)"; + + /** + * Populate organization cache with test data + * @param cache @{IgniteCache} + */ + public static void populateDataIntoOrg(IgniteCache cache) { + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + cache.put(org.getId(), org); + } + } + + /** + * Populate person cache with test data + * @param cache @{IgniteCache} + */ + public static void populateDataIntoPerson(IgniteCache cache) { + int personId = 0; + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + + for (int j = 0; j < PERSON_PER_ORG_COUNT; j++) { + Person prsn = new Person(); + prsn.setId("pers" + personId); + prsn.setOrgId(org.getId()); + prsn.setName("Person name #" + personId); + + cache.put(prsn.getId(), prsn); + + personId++; + } + } + } + + /** + * + */ + public static class Person { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String orgId; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public String getId() { + return id; + } + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getOrgId() { + return orgId; + } + + /** */ + public void setOrgId(String orgId) { + this.orgId = orgId; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + + /** + * + */ + public static class Organization { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getId() { + return id; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java new file mode 100644 index 0000000000000..62058b42260dd --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java @@ -0,0 +1,112 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import java.util.List; +import javax.cache.Cache; +import javax.cache.CacheException; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; +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.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +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.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; + +/** + * Failed to execute non-collocated query root cause message test + */ +public class NonCollocatedRetryMessageTest extends GridCommonAbstractTest { + /** */ + private static final int NODES_COUNT = 3; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + + /** */ + public void testNonCollocatedRetryMessage() { + SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + List> prsns = personCache.query(qry).getAll(); + fail("No CacheException emitted. Collection size="+prsns.size()); + } + catch (CacheException e) { + assertTrue(e.getMessage(), e.getMessage().contains("Failed to execute non-collocated query")); + } + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + volatile long reqId = -1; + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ + GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + if ( reqId < 0 ) { + reqId = req.requestId(); + String shutName = getTestIgniteInstanceName(1); + stopGrid(shutName, true, false); + } else if( reqId != req.requestId() ){ + try { + U.sleep(IgniteSystemProperties.getLong(IGNITE_SQL_RETRY_TIMEOUT, GridReduceQueryExecutor.DFLT_RETRY_TIMEOUT)); + } + catch (IgniteInterruptedCheckedException e) { + // no-op + } + } + } + } + super.sendMessage(node, msg, ackC); + } + }); + cfg.setDiscoverySpi(new TcpDiscoverySpi(){ + public long getNodesJoined() { + return stats.joinedNodesCount(); + } + }); + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + startGridsMultiThreaded(NODES_COUNT, false); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setBackups(1) + .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) + ); + final IgniteCache orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setBackups(1) + .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) + ); + + awaitPartitionMapExchange(); + + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); + } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + +} + diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java index 570d568fee212..815f9b0d06d38 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java @@ -7,9 +7,10 @@ import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.query.SqlQuery; -import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; @@ -24,14 +25,19 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; import org.apache.ignite.internal.util.GridSpinBusyLock; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +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.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; - +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.JOIN_SQL; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; /** * Test for 6 retry cases */ @@ -41,16 +47,6 @@ public class RetryCauseMessageTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 2; - - /** */ - private static final int ORG_COUNT = NODES_COUNT; - - /** */ - private static final int PERSON_PER_ORG_COUNT = 50; - /** */ - private static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + - "where Person.orgId = org.id " + - "and lower(org.name) = lower(?)"; /** */ private static final String ORG_SQL = "select * from Organization"; /** */ @@ -70,7 +66,7 @@ public class RetryCauseMessageTest extends GridCommonAbstractTest { /** * Failed to reserve partitions for query (cache is not found on local node) */ - public void testCacheWasNotFoundMessage() { + public void testSynthCacheWasNotFoundMessage() { GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", new MockGridMapQueryExecutor(null) { @@ -81,9 +77,9 @@ public void testCacheWasNotFoundMessage() { startedExecutor.onMessage(nodeId, msg); qryReq.caches().remove(qryReq.caches().size() - 1); } - else + else startedExecutor.onMessage(nodeId, msg); - + } }.insertRealExecutor(mapQryExec)); @@ -93,7 +89,7 @@ public void testCacheWasNotFoundMessage() { personCache.query(qry).getAll(); } catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + assertTrue(e.getMessage(), e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); return; } finally { @@ -339,35 +335,39 @@ public void testNonCollocatedFailureMessage() { @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setPeerClassLoadingEnabled(false); + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; - TcpDiscoverySpi disco = new TcpDiscoverySpi(); - disco.setForceServerMode(true); - disco.setIpFinder(IP_FINDER); - - cfg.setDiscoverySpi(disco); + super.sendMessage(node, msg, ackC); + } + }); return cfg; } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); GridQueryProcessor qryProc = grid(ignite.name()).context().query(); h2Idx = GridTestUtils.getFieldValue(qryProc, GridQueryProcessor.class, "idx"); personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") - .setIndexedTypes(String.class, Person.class)); + .setIndexedTypes(String.class, Person.class) + ); orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) .setCacheMode(CacheMode.REPLICATED) .setIndexedTypes(String.class, Organization.class) + // .setNodeFilter( node -> node.order()<2L ) ); awaitPartitionMapExchange(); - populateDataIntoOrg(orgCache); - populateDataIntoPerson(personCache); + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); } /** {@inheritDoc} */ @@ -375,122 +375,6 @@ public void testNonCollocatedFailureMessage() { stopAllGrids(); } - /** - * Populate organization cache with test data - * @param cache @{IgniteCache} - */ - private void populateDataIntoOrg(IgniteCache cache) { - for (int i = 0; i < ORG_COUNT; i++) { - Organization org = new Organization(); - org.setId("org" + i); - org.setName("Organization #" + i); - cache.put(org.getId(), org); - } - } - - /** - * Populate person cache with test data - * @param cache @{IgniteCache} - */ - private void populateDataIntoPerson(IgniteCache cache) { - int personId = 0; - for (int i = 0; i < ORG_COUNT; i++) { - Organization org = new Organization(); - org.setId("org" + i); - org.setName("Organization #" + i); - - for (int j = 0; j < PERSON_PER_ORG_COUNT; j++) { - Person prsn = new Person(); - prsn.setId("pers" + personId); - prsn.setOrgId(org.getId()); - prsn.setName("Person name #" + personId); - - cache.put(prsn.getId(), prsn); - - personId++; - } - } - } - - /** - * - */ - private static class Person { - /** */ - @QuerySqlField(index = true) - private String id; - - /** */ - @QuerySqlField(index = true) - private String orgId; - - /** */ - @QuerySqlField(index = true) - private String name; - - /** */ - public String getId() { - return id; - } - - /** */ - public void setId(String id) { - this.id = id; - } - - /** */ - public String getOrgId() { - return orgId; - } - - /** */ - public void setOrgId(String orgId) { - this.orgId = orgId; - } - - /** */ - public String getName() { - return name; - } - - /** */ - public void setName(String name) { - this.name = name; - } - } - - /** - * - */ - private static class Organization { - /** */ - @QuerySqlField(index = true) - private String id; - - /** */ - @QuerySqlField(index = true) - private String name; - - /** */ - public void setId(String id) { - this.id = id; - } - - /** */ - public String getId() { - return id; - } - - /** */ - public String getName() { - return name; - } - - /** */ - public void setName(String name) { - this.name = name; - } - } /** * Wrapper around @{GridMapQueryExecutor} From b006a8dba4ae4319aa78a0d3c6fd6eef47bb1da8 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Mon, 20 Aug 2018 15:29:46 +0300 Subject: [PATCH 06/95] IGNITE-9141 javadoc added --- .../processors/query/h2/twostep/ReduceQueryRun.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 2f37370cda7a9..7de2d89a360c0 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -105,6 +105,10 @@ void stateWithMsg(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { state(State.getStateBuilder().atv(msg.retry()).rootCause(msg.retryCause()).nodeId(nodeId).build()); } + /** + * + * @param state state + */ private void state(State state){ if (!this.state.compareAndSet(null, state)) return; @@ -144,20 +148,24 @@ JdbcConnection connection() { return conn; } + /** */ boolean hasError(){ return state.get()!=null; } + /** */ CacheException cacheEx() { State st = state.get(); return st!=null ? st.ex : null; } + /** */ AffinityTopologyVersion atv(){ State st = state.get(); return st!=null ? st.atv : null; } + /** */ String rootCause(){ State st = state.get(); return st!=null ? st.rootCause : null; From 02db267355a8405c64c45d51c81e342df664d612 Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 28 Aug 2018 11:45:55 +0300 Subject: [PATCH 07/95] Review comments. --- .../messages/GridQueryNextPageResponse.java | 6 +- .../h2/twostep/GridMapQueryExecutor.java | 59 ++++++++----------- .../h2/twostep/GridReduceQueryExecutor.java | 2 + .../query/h2/twostep/ReduceQueryRun.java | 6 +- .../dht/MockGridDhtLocalPartition.java | 1 + .../NonCollocatedRetryMessageTest.java | 11 +++- 6 files changed, 47 insertions(+), 38 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java index 925823c571c87..aa668df98b279 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java @@ -364,7 +364,9 @@ public void retry(AffinityTopologyVersion retry) { /** * @return Retry Ccause message. */ - public String retryCause() { return retryCause; } + public String retryCause() { + return retryCause; + } /** * @param retryCause Retry Ccause message. @@ -393,4 +395,4 @@ public void last(boolean last) { "valsSize", vals != null ? vals.size() : 0, "rowsSize", plainRows != null ? plainRows.size() : 0); } -} +} \ No newline at end of file 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 c7f2d916d3f64..ffe062b1b20b1 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 @@ -319,10 +319,9 @@ private String reservePartitions( // Cache was not found, probably was not deployed yet. if (cctx == null) { - final String res = String.format("Failed to reserve partitions for query (cache is not found on local node) [" + - "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i)); - - logRetry(res); + final String res = String.format("Failed to reserve partitions for query (cache is not found on " + + "local node) [localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", + ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i)); return res; } @@ -338,9 +337,10 @@ private String reservePartitions( if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits. if (r != MapReplicatedReservation.INSTANCE) { if (!r.reserve()) { - final String res = String.format("Failed to reserve partitions for query (group reservation failed) [" + - "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s]",ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); - logRetry(res); + final String res = String.format("Failed to reserve partitions for query (group " + + "reservation failed) [localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, " + + "cacheName=%s]",ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); + return res; } @@ -361,8 +361,9 @@ private String reservePartitions( if (partState != OWNING) { final String res = String.format("Failed to reserve partitions for query " + - "(partition of REPLICATED cache is not in OWNING state) [" + - "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + "(partition of REPLICATED cache is not in OWNING state) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + + "part=%s, partFound=%s, partState=%s]", ctx.localNodeId(), nodeId, reqId, @@ -373,7 +374,7 @@ private String reservePartitions( (part != null), partState ); - logRetry(res); + return res; } } @@ -394,8 +395,9 @@ private String reservePartitions( if (partState != OWNING || !part.reserve()) { final String res = String.format("Failed to reserve partitions for query " + - "(partition of PARTITIONED cache cannot be reserved) [" + - "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partFound=%s, partState=%s]", + "(partition of PARTITIONED cache cannot be reserved) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + + "part=%s, partFound=%s, partState=%s]", ctx.localNodeId(), nodeId, reqId, @@ -406,7 +408,7 @@ private String reservePartitions( (part != null), partState ); - logRetry(res); + return res; } @@ -418,8 +420,9 @@ private String reservePartitions( if (part.state() != OWNING) { final String res = String.format("Failed to reserve partitions for query " + - "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + - "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, part=%s, partState=%s]", + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + + "part=%s, partState=%s]", ctx.localNodeId(), nodeId, reqId, @@ -429,7 +432,7 @@ private String reservePartitions( partId, partState ); - logRetry(res); + return res; } } @@ -456,15 +459,6 @@ private String reservePartitions( return null; } - /** - * Load failed partition reservation. - * - * @param msg Message. - */ - private void logRetry(String msg) { - log.info(msg); - } - /** * @param ints Integers. * @return Collection wrapper. @@ -700,14 +694,14 @@ private void onQueryRequest0( try { if (topVer != null) { // Reserve primary for topology version or explicit partitions. - String reservationError = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); + String err = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); - if (!F.isEmpty(reservationError)) { + if (!F.isEmpty(err)) { // Unregister lazy worker because re-try may never reach this node again. if (lazy) stopAndUnregisterCurrentLazyWorker(); - sendRetry(node, reqId, segmentId, reservationError); + sendRetry(node, reqId, segmentId, err); return; } @@ -823,13 +817,11 @@ private void onQueryRequest0( if (retryErr != null) { final String retryCause = String.format( - "Failed to execute non-collocated query (will retry) [localNodeId=%s, rmtNodeId=%s, reqId=%s, errMsg=%s]", - ctx.localNodeId(), node.id(), reqId, retryErr.getMessage() + "Failed to execute non-collocated query (will retry) [localNodeId=%s, rmtNodeId=%s, reqId=%s, " + + "errMsg=%s]", ctx.localNodeId(), node.id(), reqId, retryErr.getMessage() ); - logRetry(retryCause); - - sendRetry(node, reqId, segmentId,retryCause); + sendRetry(node, reqId, segmentId, retryCause); } else { U.error(log, "Failed to execute local query.", e); @@ -878,6 +870,7 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th List reserved = new ArrayList<>(); + // TODO: Error message should be propagated back. if (!F.isEmpty(reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId))) { U.error(log, "Failed to reserve partitions for DML request. [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", reqId=" + req.requestId() + ", cacheIds=" + cacheIds + 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 47b9588ffc2af..ec4421bbe941e 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 @@ -371,6 +371,7 @@ private void retry(ReduceQueryRun r, AffinityTopologyVersion retryVer, UUID node private boolean isPreloadingActive(List cacheIds) { for (Integer cacheId : cacheIds) { + // TODO: Reformulate, e.g. "Cache not found on local node [cacheName=name]" if (null == cacheContext(cacheId)) throw new CacheException(String.format("Grid cache context is not registered for cache id=%s",cacheId)); @@ -578,6 +579,7 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); + // TODO: Better variable naming. ReduceQueryRun prevR = null; for (int attempt = 0;; attempt++) { diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 7de2d89a360c0..7ff74a30852db 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -57,7 +57,6 @@ class ReduceQueryRun { /** */ private final AtomicReference state = new AtomicReference<>(); - /** * Constructor. * @@ -85,6 +84,7 @@ class ReduceQueryRun { * @param o Fail state object. * @param nodeId Node ID. */ + // TODO: Styling void state(Object o, @Nullable UUID nodeId) { assert o != null; assert o instanceof CacheException || o instanceof AffinityTopologyVersion : o.getClass(); @@ -99,6 +99,7 @@ void state(Object o, @Nullable UUID nodeId) { * @param msg corresponding response message * @param nodeId Node ID. */ + // TODO: Styling, no abbreviations in method names void stateWithMsg(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { assert msg != null; assert msg.retry() != null; @@ -160,6 +161,7 @@ CacheException cacheEx() { } /** */ + // TODO: atv -> topVer AffinityTopologyVersion atv(){ State st = state.get(); return st!=null ? st.atv : null; @@ -205,6 +207,7 @@ private static StateBuilder getStateBuilder(){ private final String rootCause; /** */ + // TODO: atv -> topVer private final AffinityTopologyVersion atv; /** */ @@ -220,6 +223,7 @@ private State(CacheException ex, String rootCause, AffinityTopologyVersion atv, } /** */ + // TODO: We do not need this. private static class StateBuilder{ /** */ private CacheException ex = null; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java index 33d7613c9c99a..2a0dcbe449ef2 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java @@ -22,6 +22,7 @@ /** * Wrapper around GridDhtLocalPartition to be extended in test cases */ +// TODO: Move to H2 tests public abstract class MockGridDhtLocalPartition extends GridDhtLocalPartition{ /** * fake id generator in order to bypass construction stage failure (if invoked with real id) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java index 62058b42260dd..b0aa3840842f0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java @@ -25,11 +25,15 @@ /** * Failed to execute non-collocated query root cause message test */ +// TODO: Test -> SelfTest +// TODO: Add to suite. public class NonCollocatedRetryMessageTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 3; + /** */ private static final String ORG = "org"; + /** */ private IgniteCache personCache; @@ -58,11 +62,14 @@ public void testNonCollocatedRetryMessage() { GridIoMessage gridMsg = (GridIoMessage)msg; if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); - if ( reqId < 0 ) { + if (reqId < 0) { reqId = req.requestId(); + String shutName = getTestIgniteInstanceName(1); + stopGrid(shutName, true, false); - } else if( reqId != req.requestId() ){ + } + else if( reqId != req.requestId() ){ try { U.sleep(IgniteSystemProperties.getLong(IGNITE_SQL_RETRY_TIMEOUT, GridReduceQueryExecutor.DFLT_RETRY_TIMEOUT)); } From 0c4301cdc0d6108ed5b51173144e19d3ad450e63 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 28 Aug 2018 14:08:47 +0300 Subject: [PATCH 08/95] IGNITE-9141 Fixes according to review --- .../h2/twostep/GridMapQueryExecutor.java | 40 +-- .../h2/twostep/GridReduceQueryExecutor.java | 21 +- .../query/h2/twostep/ReduceQueryRun.java | 99 ++---- .../dht/MockGridDhtLocalPartition.java | 311 ------------------ ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 1 + ...pearedCacheCauseRetryMessageSelfTest.java} | 36 +- ...earedCacheWasNotFoundMessageSelfTest.java} | 35 +- .../query/h2/twostep/JoinSqlTestHelper.java | 35 +- ...=> NonCollocatedRetryMessageSelfTest.java} | 33 +- ...st.java => RetryCauseMessageSelfTest.java} | 157 ++++----- .../IgniteCacheQuerySelfTestSuite2.java | 9 + 11 files changed, 243 insertions(+), 534 deletions(-) delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java rename modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/{DisappearedCacheCauseRetryMessageTest.java => DisappearedCacheCauseRetryMessageSelfTest.java} (80%) rename modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/{DisappearedCacheWasNotFoundMessageTest.java => DisappearedCacheWasNotFoundMessageSelfTest.java} (80%) rename modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/{NonCollocatedRetryMessageTest.java => NonCollocatedRetryMessageSelfTest.java} (84%) rename modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/{RetryCauseMessageTest.java => RetryCauseMessageSelfTest.java} (75%) 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 ffe062b1b20b1..4f09b23676697 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 @@ -336,14 +336,11 @@ private String reservePartitions( if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits. if (r != MapReplicatedReservation.INSTANCE) { - if (!r.reserve()) { - final String res = String.format("Failed to reserve partitions for query (group " + + if (!r.reserve()) + return String.format("Failed to reserve partitions for query (group " + "reservation failed) [localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, " + "cacheName=%s]",ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i), cctx.name()); - return res; - } - reserved.add(r); } } @@ -358,9 +355,8 @@ private String reservePartitions( // We don't need to reserve partitions because they will not be evicted in replicated caches. GridDhtPartitionState partState = part != null ? part.state() : null; - if (partState != OWNING) { - - final String res = String.format("Failed to reserve partitions for query " + + if (partState != OWNING) + return String.format("Failed to reserve partitions for query " + "(partition of REPLICATED cache is not in OWNING state) [" + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + "part=%s, partFound=%s, partState=%s]", @@ -374,9 +370,6 @@ private String reservePartitions( (part != null), partState ); - - return res; - } } // Mark that we checked this replicated cache. @@ -392,9 +385,8 @@ private String reservePartitions( GridDhtPartitionState partState = part != null ? part.state() : null; - if (partState != OWNING || !part.reserve()) { - - final String res = String.format("Failed to reserve partitions for query " + + if (partState != OWNING || !part.reserve()) + return String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache cannot be reserved) [" + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + "part=%s, partFound=%s, partState=%s]", @@ -409,17 +401,13 @@ private String reservePartitions( partState ); - return res; - } - reserved.add(part); // Double check that we are still in owning state and partition contents are not cleared. partState = part.state(); - if (part.state() != OWNING) { - - final String res = String.format("Failed to reserve partitions for query " + + if (part.state() != OWNING) + return String.format("Failed to reserve partitions for query " + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + "part=%s, partState=%s]", @@ -432,9 +420,6 @@ private String reservePartitions( partId, partState ); - - return res; - } } if (explicitParts == null) { @@ -870,14 +855,15 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th List reserved = new ArrayList<>(); - // TODO: Error message should be propagated back. - if (!F.isEmpty(reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId))) { + String err = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); + + if (!F.isEmpty(err)) { U.error(log, "Failed to reserve partitions for DML request. [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", reqId=" + req.requestId() + ", cacheIds=" + cacheIds + ", topVer=" + topVer + ", parts=" + Arrays.toString(parts) + ']'); - sendUpdateResponse(node, reqId, null, "Failed to reserve partitions for DML request. " + - "Explanation (Retry your request when re-balancing is over)."); + sendUpdateResponse(node, reqId, null, + "Failed to reserve partitions for DML request. " + err); 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 ec4421bbe941e..8fe9c7ce516dc 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 @@ -38,7 +38,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import javax.cache.CacheException; @@ -311,7 +310,7 @@ private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse @Override public void fetchNextPage() { if (r.hasError()) { - CacheException err0 = r.cacheEx(); + CacheException err0 = r.cacheException(); if (err0 != null && err0.getCause() instanceof IgniteClientDisconnectedException) throw err0; @@ -350,7 +349,7 @@ private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse idx.addPage(page); if (msg.retry() != null) - r.stateWithMsg(msg, node.id()); + r.stateWithMessage(msg, node.id()); else if (msg.page() == 0) // Do count down on each first page received. r.latch().countDown(); } @@ -371,9 +370,8 @@ private void retry(ReduceQueryRun r, AffinityTopologyVersion retryVer, UUID node private boolean isPreloadingActive(List cacheIds) { for (Integer cacheId : cacheIds) { - // TODO: Reformulate, e.g. "Cache not found on local node [cacheName=name]" if (null == cacheContext(cacheId)) - throw new CacheException(String.format("Grid cache context is not registered for cache id=%s",cacheId)); + throw new CacheException(String.format("Cache not found on local node [cacheId=%d]",cacheId)); if (hasMovingPartitions(cacheContext(cacheId))) return true; @@ -579,12 +577,11 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); - // TODO: Better variable naming. - ReduceQueryRun prevR = null; + ReduceQueryRun lastRun = null; for (int attempt = 0;; attempt++) { if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) { - String rcValue = prevR.rootCause(); + String rcValue = lastRun.rootCause(); throw new CacheException((!F.isEmpty(rcValue))?rcValue:("Failed to map SQL query to topology.")); } if (attempt != 0) { @@ -789,8 +786,8 @@ public Iterator> query( awaitAllReplies(r, nodes, cancel); if (r.hasError()) { - if (r.cacheEx() != null) { - CacheException err = r.cacheEx(); + if (r.cacheException() != null) { + CacheException err = r.cacheException(); if (err.getCause() instanceof IgniteClientDisconnectedException) throw err; @@ -803,7 +800,7 @@ public Iterator> query( retry = true; // If remote node asks us to retry then we have outdated full partition map. - h2.awaitForReadyTopologyVersion(r.atv()); + h2.awaitForReadyTopologyVersion(r.topVersion()); } } } @@ -851,7 +848,7 @@ public Iterator> query( if (retry) { assert r != null; - prevR=r; + lastRun=r; if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 7ff74a30852db..0fee68c7ea480 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -17,6 +17,13 @@ package org.apache.ignite.internal.processors.query.h2.twostep; +import java.sql.Connection; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; +import javax.cache.CacheException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.query.GridQueryCancel; @@ -25,14 +32,6 @@ import org.h2.jdbc.JdbcConnection; import org.jetbrains.annotations.Nullable; -import javax.cache.CacheException; -import java.sql.Connection; -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicReference; - import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS; /** @@ -84,26 +83,28 @@ class ReduceQueryRun { * @param o Fail state object. * @param nodeId Node ID. */ - // TODO: Styling void state(Object o, @Nullable UUID nodeId) { assert o != null; + assert o instanceof CacheException || o instanceof AffinityTopologyVersion : o.getClass(); - StateBuilder sb = State.getStateBuilder().nodeId(nodeId); + if ( o instanceof CacheException ) - sb.exception((CacheException)o); - else sb.atv( (AffinityTopologyVersion)o); - state(sb.build()); + state(new State((CacheException)o, null, null, nodeId)); + + else + state(new State(null, null, (AffinityTopologyVersion)o, nodeId)); } /** * @param msg corresponding response message * @param nodeId Node ID. */ - // TODO: Styling, no abbreviations in method names - void stateWithMsg(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { + void stateWithMessage(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { assert msg != null; + assert msg.retry() != null; - state(State.getStateBuilder().atv(msg.retry()).rootCause(msg.retryCause()).nodeId(nodeId).build()); + + state(new State(null, msg.retryCause(), msg.retry(), nodeId)); } /** @@ -155,21 +156,23 @@ boolean hasError(){ } /** */ - CacheException cacheEx() { + CacheException cacheException() { State st = state.get(); + return st!=null ? st.ex : null; } /** */ - // TODO: atv -> topVer - AffinityTopologyVersion atv(){ + AffinityTopologyVersion topVersion(){ State st = state.get(); - return st!=null ? st.atv : null; + + return st!=null ? st.topVer : null; } /** */ String rootCause(){ State st = state.get(); + return st!=null ? st.rootCause : null; } /** @@ -196,10 +199,6 @@ void latch(CountDownLatch latch) { /** */ private static class State{ - /** */ - private static StateBuilder getStateBuilder(){ - return new StateBuilder(); - } /** */ private final CacheException ex; @@ -207,64 +206,20 @@ private static StateBuilder getStateBuilder(){ private final String rootCause; /** */ - // TODO: atv -> topVer - private final AffinityTopologyVersion atv; + private final AffinityTopologyVersion topVer; /** */ private final UUID nodeId; /** */ - private State(CacheException ex, String rootCause, AffinityTopologyVersion atv, UUID nodeId){ + private State(CacheException ex, String rootCause, AffinityTopologyVersion topVer, UUID nodeId){ this.ex=ex; - this.rootCause = rootCause; - this.atv = atv; - this.nodeId = nodeId; - } - } - /** */ - // TODO: We do not need this. - private static class StateBuilder{ - /** */ - private CacheException ex = null; - - /** */ - private String rootCause = null; - - /** */ - private AffinityTopologyVersion atv = null; - - /** */ - private UUID nodeId = null; - - /** */ - private State build(){ - return new State(ex, rootCause, atv, nodeId); - } - - /** */ - private StateBuilder exception(CacheException ex){ - this.ex = ex; - return this; - } - - /** */ - private StateBuilder rootCause(String rootCause){ - this.rootCause=rootCause; - return this; - } + this.rootCause = rootCause; - /** */ - private StateBuilder atv(AffinityTopologyVersion atv){ - this.atv=atv; - return this; - } + this.topVer = topVer; - /** */ - private StateBuilder nodeId(UUID nodeId){ this.nodeId = nodeId; - return this; } - } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java deleted file mode 100644 index 2a0dcbe449ef2..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java +++ /dev/null @@ -1,311 +0,0 @@ -package org.apache.ignite.internal.processors.cache.distributed.dht; - -import java.util.Collection; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.NodeStoppingException; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; -import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; -import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; -import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.lang.IgniteInClosure; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -/** - * Wrapper around GridDhtLocalPartition to be extended in test cases - */ -// TODO: Move to H2 tests -public abstract class MockGridDhtLocalPartition extends GridDhtLocalPartition{ - /** - * fake id generator in order to bypass construction stage failure (if invoked with real id) - */ - private static AtomicInteger cntr = new AtomicInteger(Integer.MAX_VALUE); - - /** - * Real object - */ - private GridDhtLocalPartition internal; - - /** - * @param ctx Context. - * @param grp Cache group. - * @param id Partition ID. - */ - private MockGridDhtLocalPartition(GridCacheSharedContext ctx, - CacheGroupContext grp, int id) { - super(ctx, grp, id); - } - - /** */ - protected MockGridDhtLocalPartition(GridCacheSharedContext ctx, - CacheGroupContext grp, GridDhtLocalPartition internal){ - this(ctx, grp, cntr.getAndDecrement()); - this.internal = internal; - } - - /** */ - protected GridDhtLocalPartition getInternal(){ - return internal; - } - - /** {@inheritDoc} */ - @Override public int internalSize() { - return internal.internalSize(); - } - - /** {@inheritDoc} */ - @Override protected CacheMapHolder entriesMap(GridCacheContext cctx) { - return internal.entriesMap(cctx); - } - - /** {@inheritDoc} */ - @Nullable @Override protected CacheMapHolder entriesMapIfExists(Integer cacheId) { - return internal.entriesMapIfExists(cacheId); - } - - /** {@inheritDoc} */ - @Override public IgniteCacheOffheapManager.CacheDataStore dataStore() { - return internal.dataStore(); - } - - /** {@inheritDoc} */ - @Override public boolean addReservation(GridDhtPartitionsReservation r) { - return internal.addReservation(r); - } - - /** {@inheritDoc} */ - @Override public void removeReservation(GridDhtPartitionsReservation r) { - internal.removeReservation(r); - } - - /** {@inheritDoc} */ - @Override public int id() { - return internal.id(); - } - - /** {@inheritDoc} */ - @Override public GridDhtPartitionState state() { - return internal.state(); - } - - /** {@inheritDoc} */ - @Override public int reservations() { - return internal.reservations(); - } - - /** {@inheritDoc} */ - @Override public boolean isEmpty() { - return internal.isEmpty(); - } - - /** {@inheritDoc} */ - @Override public boolean valid() { - return internal.valid(); - } - - /** {@inheritDoc} */ - @Override public void cleanupRemoveQueue() { - internal.cleanupRemoveQueue(); - } - - /** {@inheritDoc} */ - @Override public void onDeferredDelete(int cacheId, KeyCacheObject key, GridCacheVersion ver) { - internal.onDeferredDelete(cacheId,key,ver); - } - - /** {@inheritDoc} */ - @Override public void lock() { - internal.lock(); - } - - /** {@inheritDoc} */ - @Override public void unlock() { - internal.unlock(); - } - - /** {@inheritDoc} */ - @Override public boolean reserve() { - return internal.reserve(); - } - - /** {@inheritDoc} */ - @Override public void release() { - internal.release(); - } - - /** {@inheritDoc} */ - @Override protected void release(int sizeChange, CacheMapHolder hld, GridCacheEntryEx e) { - internal.release(); - } - - /** {@inheritDoc} */ - @Override public void restoreState(GridDhtPartitionState stateToRestore) { - internal.restoreState(stateToRestore); - } - - /** {@inheritDoc} */ - @Override public void moving() { - - internal.moving(); - } - - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture rent(boolean updateSeq) { - return internal.rent(updateSeq); - } - - /** {@inheritDoc} */ - @Override public void clearAsync() { - internal.clearAsync(); - } - - /** {@inheritDoc} */ - @Override public boolean markForDestroy() { - return internal.markForDestroy(); - } - - /** {@inheritDoc} */ - @Override public void destroy() { - internal.destroy(); - } - - - /** {@inheritDoc} */ - @Override public void awaitDestroy() { - internal.awaitDestroy(); - } - - - /** {@inheritDoc} */ - @Override public void onClearFinished(IgniteInClosure> lsnr) { - internal.onClearFinished(lsnr); - } - - - /** {@inheritDoc} */ - @Override public boolean isClearing() { - return internal.isClearing(); - } - - - /** {@inheritDoc} */ - @Override public boolean tryClear(EvictionContext evictionCtx) throws NodeStoppingException { - return internal.tryClear(evictionCtx); - } - - /** {@inheritDoc} */ - @Override public boolean primary(AffinityTopologyVersion topVer) { - return internal.primary(topVer); - } - - - /** {@inheritDoc} */ - @Override public boolean backup(AffinityTopologyVersion topVer) { - return internal.backup(topVer); - } - - /** {@inheritDoc} */ - @Override public long initialUpdateCounter() { - return internal.initialUpdateCounter(); - } - - - /** {@inheritDoc} */ - @Override public void updateCounter(long val) { - - internal.updateCounter(val); - } - - - /** {@inheritDoc} */ - @Override public void initialUpdateCounter(long val) { - - internal.initialUpdateCounter(val); - } - - - /** {@inheritDoc} */ - @Override public long fullSize() { - - return internal.fullSize(); - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - - return internal.hashCode(); - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object obj) { - - return internal.equals(obj); - } - - /** {@inheritDoc} */ - @Override public int compareTo(@NotNull GridDhtLocalPartition part) { - return internal.compareTo(part); - } - - /** {@inheritDoc} */ - @Override public String toString() { - - return internal.toString(); - } - - /** {@inheritDoc} */ - @Override public int publicSize(int cacheId) { - - return internal.publicSize(cacheId); - } - - /** {@inheritDoc} */ - @Override public void incrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { - internal.incrementPublicSize(hld,e); - } - - /** {@inheritDoc} */ - @Override public void decrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { - internal.decrementPublicSize(hld,e); - } - - /** {@inheritDoc} */ - @Nullable @Override public GridCacheMapEntry getEntry(GridCacheContext ctx, KeyCacheObject key) { - return internal.getEntry(ctx,key); - } - - /** {@inheritDoc} */ - @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent( - GridCacheContext ctx, - final AffinityTopologyVersion topVer, - KeyCacheObject key, - final boolean create, - final boolean touch) { - return internal.putEntryIfObsoleteOrAbsent(ctx, topVer, key, create, touch); - } - - /** {@inheritDoc} */ - @Override public boolean removeEntry(final GridCacheEntryEx entry) { - - return internal.removeEntry(entry); - } - - /** {@inheritDoc} */ - @Override public Collection entries(int cacheId, final CacheEntryPredicate... filter) { - return internal.entries(cacheId, filter); - } - - /** {@inheritDoc} */ - @Override public Set entrySet(int cacheId, final CacheEntryPredicate... filter) { - return internal.entrySet(cacheId, filter); - } -} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 56fd7b8d13b75..a0cea727cc6b0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -216,6 +216,7 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni try(QueryCursor> ignored = cursor) { cursor.iterator(); + fail(); } catch (CacheException ex) { log().error("Got expected exception", ex); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageSelfTest.java similarity index 80% rename from modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageSelfTest.java index 0072fe4a0a79a..8c4358a7fd722 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageSelfTest.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.query.h2.twostep; import javax.cache.CacheException; @@ -23,8 +40,7 @@ /** * Failed to reserve partitions for query (cache is not found on local node) Root cause test */ -public class DisappearedCacheCauseRetryMessageTest extends GridCommonAbstractTest { - +public class DisappearedCacheCauseRetryMessageSelfTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 2; /** */ @@ -34,12 +50,16 @@ public class DisappearedCacheCauseRetryMessageTest extends GridCommonAbstractTes /** */ private IgniteCache orgCache; + /** */ public void testDisappearedCacheCauseRetryMessage() { SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { personCache.query(qry).getAll(); + fail("No CacheException emitted."); } catch (CacheException e) { @@ -57,23 +77,27 @@ public void testDisappearedCacheCauseRetryMessage() { /** {@inheritDoc} */ @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); reqId = req.requestId(); orgCache.destroy(); - } else if ( GridQueryCancelRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + } + else if ( GridQueryCancelRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ GridQueryCancelRequest req = (GridQueryCancelRequest) (gridMsg.message()); if (reqId == req.queryRequestId()) - orgCache = DisappearedCacheCauseRetryMessageTest.this.ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + orgCache = DisappearedCacheCauseRetryMessageSelfTest.this.ignite(0).getOrCreateCache(new CacheConfiguration(ORG) .setCacheMode(CacheMode.REPLICATED) .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) ); } } + super.sendMessage(node, msg, ackC); } }); @@ -84,11 +108,13 @@ public void testDisappearedCacheCauseRetryMessage() { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + startGridsMultiThreaded(NODES_COUNT, false); personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) ); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) .setCacheMode(CacheMode.REPLICATED) .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) @@ -97,8 +123,10 @@ public void testDisappearedCacheCauseRetryMessage() { awaitPartitionMapExchange(); JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java similarity index 80% rename from modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java index ce106a8136877..91af36e72f85c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.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.query.h2.twostep; import javax.cache.CacheException; @@ -23,8 +40,7 @@ /** * Grid cache context is not registered for cache id root cause message test */ -public class DisappearedCacheWasNotFoundMessageTest extends GridCommonAbstractTest { - +public class DisappearedCacheWasNotFoundMessageSelfTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 2; /** */ @@ -34,12 +50,15 @@ public class DisappearedCacheWasNotFoundMessageTest extends GridCommonAbstractTe /** */ private IgniteCache orgCache; + /** */ public void testDisappearedCacheWasNotFoundMessage() { - SqlQuery qry = new SqlQuery(Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { personCache.query(qry).getAll(); + fail("No CacheException emitted."); } catch (CacheException e) { @@ -55,14 +74,19 @@ public void testDisappearedCacheWasNotFoundMessage() { /** {@inheritDoc} */ @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + req.requestId(); + orgCache.destroy(); } } + super.sendMessage(node, msg, ackC); } }); @@ -73,11 +97,13 @@ public void testDisappearedCacheWasNotFoundMessage() { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); - Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); + + startGridsMultiThreaded(NODES_COUNT, false); personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) ); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) .setCacheMode(CacheMode.REPLICATED) .setIndexedTypes(String.class, Organization.class) @@ -86,6 +112,7 @@ public void testDisappearedCacheWasNotFoundMessage() { awaitPartitionMapExchange(); JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java index f9c3c481487e9..2abab7448b166 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.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.query.h2.twostep; import org.apache.ignite.IgniteCache; @@ -7,14 +24,12 @@ * Join sql test helper */ public class JoinSqlTestHelper { - /** */ private static final int ORG_COUNT = 100; - /** */ private static final int PERSON_PER_ORG_COUNT = 10; /** */ - public static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + + static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + "where Person.orgId = org.id " + "and lower(org.name) = lower(?)"; @@ -22,11 +37,14 @@ public class JoinSqlTestHelper { * Populate organization cache with test data * @param cache @{IgniteCache} */ - public static void populateDataIntoOrg(IgniteCache cache) { + static void populateDataIntoOrg(IgniteCache cache) { for (int i = 0; i < ORG_COUNT; i++) { Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + cache.put(org.getId(), org); } } @@ -35,17 +53,23 @@ public static void populateDataIntoOrg(IgniteCache cache) * Populate person cache with test data * @param cache @{IgniteCache} */ - public static void populateDataIntoPerson(IgniteCache cache) { + static void populateDataIntoPerson(IgniteCache cache) { int personId = 0; + for (int i = 0; i < ORG_COUNT; i++) { Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); for (int j = 0; j < PERSON_PER_ORG_COUNT; j++) { Person prsn = new Person(); + prsn.setId("pers" + personId); + prsn.setOrgId(org.getId()); + prsn.setName("Person name #" + personId); cache.put(prsn.getId(), prsn); @@ -134,5 +158,4 @@ public void setName(String name) { this.name = name; } } - } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.java similarity index 84% rename from modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.java index b0aa3840842f0..c602225e8e30e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageSelfTest.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.query.h2.twostep; import java.util.List; @@ -25,9 +42,7 @@ /** * Failed to execute non-collocated query root cause message test */ -// TODO: Test -> SelfTest -// TODO: Add to suite. -public class NonCollocatedRetryMessageTest extends GridCommonAbstractTest { +public class NonCollocatedRetryMessageSelfTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 3; @@ -40,7 +55,9 @@ public class NonCollocatedRetryMessageTest extends GridCommonAbstractTest { /** */ public void testNonCollocatedRetryMessage() { SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { List> prsns = personCache.query(qry).getAll(); fail("No CacheException emitted. Collection size="+prsns.size()); @@ -53,15 +70,19 @@ public void testNonCollocatedRetryMessage() { /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ volatile long reqId = -1; /** {@inheritDoc} */ @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + if (reqId < 0) { reqId = req.requestId(); @@ -82,23 +103,27 @@ else if( reqId != req.requestId() ){ super.sendMessage(node, msg, ackC); } }); + cfg.setDiscoverySpi(new TcpDiscoverySpi(){ public long getNodesJoined() { return stats.joinedNodesCount(); } }); + return cfg; } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + startGridsMultiThreaded(NODES_COUNT, false); personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") .setBackups(1) .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) ); + final IgniteCache orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) .setBackups(1) .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) @@ -107,8 +132,10 @@ public long getNodesJoined() { awaitPartitionMapExchange(); JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java similarity index 75% rename from modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java index 815f9b0d06d38..1ead3aa5e5837 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java @@ -1,9 +1,25 @@ +/* + * 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.twostep; import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReferenceArray; import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -16,10 +32,7 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; -import org.apache.ignite.internal.processors.cache.distributed.dht.MockGridDhtLocalPartition; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; @@ -28,23 +41,19 @@ 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.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; -import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.JOIN_SQL; import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; + /** * Test for 6 retry cases */ -public class RetryCauseMessageTest extends GridCommonAbstractTest { - /** IP finder. */ - private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); - +public class RetryCauseMessageSelfTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 2; /** */ @@ -68,28 +77,34 @@ public class RetryCauseMessageTest extends GridCommonAbstractTest { */ public void testSynthCacheWasNotFoundMessage() { GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", new MockGridMapQueryExecutor(null) { @Override public void onMessage(UUID nodeId, Object msg) { if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + qryReq.caches().add(Integer.MAX_VALUE); + startedExecutor.onMessage(nodeId, msg); + qryReq.caches().remove(qryReq.caches().size() - 1); } else startedExecutor.onMessage(nodeId, msg); - } }.insertRealExecutor(mapQryExec)); SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { personCache.query(qry).getAll(); } catch (CacheException e) { assertTrue(e.getMessage(), e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + return; } finally { @@ -103,6 +118,7 @@ public void testSynthCacheWasNotFoundMessage() { */ public void testGrpReservationFailureMessage() { final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", @@ -110,26 +126,30 @@ public void testGrpReservationFailureMessage() { @Override public void onMessage(UUID nodeId, Object msg) { if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { return false; } + @Override public void release() {} - }); } startedExecutor.onMessage(nodeId, msg); - } }.insertRealExecutor(mapQryExec)); SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { personCache.query(qry).getAll(); } catch (CacheException e) { assertTrue(e.getMessage().contains("Failed to reserve partitions for query (group reservation failed) [")); + return; } finally { @@ -143,17 +163,25 @@ public void testGrpReservationFailureMessage() { */ public void testReplicatedCacheReserveFailureMessage() { GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", new MockGridMapQueryExecutor(null) { @Override public void onMessage(UUID nodeId, Object msg) { if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); } else @@ -162,12 +190,14 @@ public void testReplicatedCacheReserveFailureMessage() { }.insertRealExecutor(mapQryExec)); SqlQuery qry = new SqlQuery<>(Organization.class, ORG_SQL); + qry.setDistributedJoins(true); try { orgCache.query(qry).getAll(); } catch (CacheException e) { assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) [")); + return; } finally { @@ -181,17 +211,25 @@ public void testReplicatedCacheReserveFailureMessage() { */ public void testPartitionedCacheReserveFailureMessage() { GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", new MockGridMapQueryExecutor(null) { @Override public void onMessage(UUID nodeId, Object msg) { if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); } else @@ -201,87 +239,14 @@ public void testPartitionedCacheReserveFailureMessage() { }.insertRealExecutor(mapQryExec)); SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); try { personCache.query(qry).getAll(); } catch (CacheException e) { assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) [")); - return; - } - finally { - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); - } - fail(); - } - - /** - * Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) - */ - public void testPartitionStateChangedMessage() { - final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); - final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); - final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", - new MockGridMapQueryExecutor(null) { - @Override public void onMessage(UUID nodeId, Object msg) { - if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { - GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; - final MapReservationKey grpKey = new MapReservationKey(ORG, null); - reservations.put(grpKey, new GridReservable() { - @Override public boolean reserve() { - return true; - } - @Override public void release() { - - } - }); - GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); - - if (GridDhtPartitionTopologyImpl.class.isAssignableFrom(cctx.topology().getClass())) { - GridDhtPartitionTopologyImpl tpg = (GridDhtPartitionTopologyImpl)(cctx.topology()); - AtomicReferenceArray locParts = GridTestUtils.getFieldValue(tpg, GridDhtPartitionTopologyImpl.class, "locParts"); - GridDhtLocalPartition part = locParts.get(0); - MockGridDhtLocalPartition mockPart = new MockGridDhtLocalPartition( - GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "ctx"), - GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "grp"), - part) { - volatile private boolean preReserved = true; - - @Override public boolean reserve() { - preReserved = false; - return super.reserve(); - } - - @Override public GridDhtPartitionState state() { - if (preReserved) - return super.state(); - else - return MOVING; - } - }; - locParts.set(0, mockPart); - startedExecutor.onMessage(nodeId, msg); - locParts.set(0, part); - } - else - startedExecutor.onMessage(nodeId, msg); - - } - else - startedExecutor.onMessage(nodeId, msg); - - } - }.insertRealExecutor(mapQryExec)); - - SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - } - catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) [")); return; } finally { @@ -295,19 +260,22 @@ public void testPartitionStateChangedMessage() { */ public void testNonCollocatedFailureMessage() { final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", new MockGridMapQueryExecutor(null) { @Override public void onMessage(UUID nodeId, Object msg) { if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { throw new GridH2RetryException("test retry exception"); } @Override public void release() { - } }); } @@ -317,12 +285,14 @@ public void testNonCollocatedFailureMessage() { }.insertRealExecutor(mapQryExec)); SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); try { personCache.query(qry).getAll(); } catch (CacheException e) { assertTrue(e.getMessage().contains("Failed to execute non-collocated query (will retry) [")); + return; } finally { @@ -350,7 +320,9 @@ public void testNonCollocatedFailureMessage() { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); + GridQueryProcessor qryProc = grid(ignite.name()).context().query(); h2Idx = GridTestUtils.getFieldValue(qryProc, GridQueryProcessor.class, "idx"); @@ -358,15 +330,16 @@ public void testNonCollocatedFailureMessage() { personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") .setIndexedTypes(String.class, Person.class) ); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) .setCacheMode(CacheMode.REPLICATED) .setIndexedTypes(String.class, Organization.class) - // .setNodeFilter( node -> node.order()<2L ) ); awaitPartitionMapExchange(); JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); } @@ -401,26 +374,21 @@ MockGridMapQueryExecutor insertRealExecutor(GridMapQueryExecutor realExecutor) { /** {@inheritDoc} */ @Override public void onMessage(UUID nodeId, Object msg) { - startedExecutor.onMessage(nodeId, msg); } /** {@inheritDoc} */ @Override public void cancelLazyWorkers() { - startedExecutor.cancelLazyWorkers(); } /** {@inheritDoc} */ @Override GridSpinBusyLock busyLock() { - return startedExecutor.busyLock(); - } /** {@inheritDoc} */ @Override public void onCacheStop(String cacheName) { - startedExecutor.onCacheStop(cacheName); } @@ -436,7 +404,6 @@ MockGridMapQueryExecutor insertRealExecutor(GridMapQueryExecutor realExecutor) { /** {@inheritDoc} */ @Override public int registeredLazyWorkers() { - return startedExecutor.registeredLazyWorkers(); } } 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 093423dd50f0d..536834cda9346 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 @@ -51,6 +51,10 @@ import org.apache.ignite.internal.processors.query.IgniteCacheGroupsSqlSegmentedIndexMultiNodeSelfTest; import org.apache.ignite.internal.processors.query.IgniteCacheGroupsSqlSegmentedIndexSelfTest; import org.apache.ignite.internal.processors.query.h2.twostep.CacheQueryMemoryLeakTest; +import org.apache.ignite.internal.processors.query.h2.twostep.DisappearedCacheCauseRetryMessageSelfTest; +import org.apache.ignite.internal.processors.query.h2.twostep.DisappearedCacheWasNotFoundMessageSelfTest; +import org.apache.ignite.internal.processors.query.h2.twostep.NonCollocatedRetryMessageSelfTest; +import org.apache.ignite.internal.processors.query.h2.twostep.RetryCauseMessageSelfTest; import org.apache.ignite.testframework.IgniteTestSuite; /** @@ -110,6 +114,11 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheQueryMemoryLeakTest.class); + suite.addTestSuite(NonCollocatedRetryMessageSelfTest.class); + suite.addTestSuite(RetryCauseMessageSelfTest.class); + suite.addTestSuite(DisappearedCacheCauseRetryMessageSelfTest.class); + suite.addTestSuite(DisappearedCacheWasNotFoundMessageSelfTest.class); + return suite; } } From dd35f024d77d12badf711bce3644450008e38921 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Wed, 29 Aug 2018 15:15:43 +0300 Subject: [PATCH 09/95] IGNITE-8913 Query cancelled messages are enriched with details, tests updated --- .../cache/query/QueryCancelledException.java | 11 +- .../processors/cache/QueryCursorImpl.java | 5 +- .../processors/query/h2/IgniteH2Indexing.java | 8 +- .../h2/twostep/GridMapQueryExecutor.java | 13 ++- .../h2/twostep/GridReduceQueryExecutor.java | 109 +++++++++++------- .../query/h2/twostep/ReduceQueryRun.java | 8 +- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 92 ++++++++++----- ...acheLocalQueryCancelOrTimeoutSelfTest.java | 23 +++- 8 files changed, 182 insertions(+), 87 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java index 5f5ffdce163fe..3e0662f08e9cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java @@ -20,7 +20,7 @@ import org.apache.ignite.IgniteCheckedException; /** - * The exception is thrown if a query was cancelled or timed out while executing. + * The exception is thrown if a qry was cancelled or timed out while executing. */ public class QueryCancelledException extends IgniteCheckedException { /** */ @@ -30,6 +30,11 @@ public class QueryCancelledException extends IgniteCheckedException { * Default constructor. */ public QueryCancelledException() { - super("The query was cancelled while executing."); + super("The qry was cancelled while executing"); } -} \ No newline at end of file + + /** */ + public QueryCancelledException(String msg) { + super(msg); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java index 4fc1054cd69e0..9cb0a130f9d20 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java @@ -98,7 +98,10 @@ public QueryCursorImpl(Iterable iterExec, GridQueryCancel cancel, boolean isQ // Handle race with cancel and make sure the iterator resources are freed correctly. closeIter(); - throw new CacheException(new QueryCancelledException()); + throw new CacheException(new QueryCancelledException(String.format( + "The query was cancelled while executing. Client node should provide details [reason=%s]", + "Cancelled by client" + ))); } assert iter != null; 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 6b7eddfba54f2..bc4dfdbd42006 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 @@ -1231,7 +1231,13 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement catch (SQLException e) { // Throw special exception. if (e.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED) - throw new QueryCancelledException(); + throw new CacheException(new QueryCancelledException(String.format( + "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", + stmt, + ctx.localNodeId(), + timeoutMillis>0 ? "Statement with timeout was cancelled" : "Cancelled by client", + timeoutMillis + ))); throw new IgniteCheckedException("Failed to execute SQL query. " + e.getMessage(), e); } 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 4f09b23676697..d348e21126208 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 @@ -726,7 +726,14 @@ private void onQueryRequest0( nodeRess.cancelRequest(reqId); - throw new QueryCancelledException(); + throw new QueryCancelledException(String.format( + "The query request (could be more than 1 query) was cancelled while executing. " + + "[reqId=%s, firstQuery=%s, localNodeId=%s, reason=%s]", + reqId, + qrys.isEmpty() ? "no queries" : qrys.iterator().next().query(), + ctx.localNodeId(), + "Cancelled by client" + )); } // Run queries. @@ -811,7 +818,9 @@ private void onQueryRequest0( else { U.error(log, "Failed to execute local query.", e); - sendError(node, reqId, e); + Exception cancelled = X.cause(e,QueryCancelledException.class); + + sendError(node, reqId, (cancelled != null) ? cancelled : e); if (e instanceof Error) throw (Error)e; 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 8fe9c7ce516dc..2e23a7e0abac0 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 @@ -281,7 +281,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { if (failCode == GridQueryFailResponse.CANCELLED_BY_ORIGINATOR) e.addSuppressed(new QueryCancelledException()); - r.state(e, nodeId); + r.state(msg, e, nodeId); } } @@ -360,7 +360,7 @@ else if (msg.page() == 0) // Do count down on each first page received. * @param nodeId Node ID. */ private void retry(ReduceQueryRun r, AffinityTopologyVersion retryVer, UUID nodeId) { - r.state(retryVer, nodeId); + r.state("Node left the grid", retryVer, nodeId); } /** @@ -751,24 +751,6 @@ public Iterator> query( boolean retry = false; - // Always enforce join order on map side to have consistent behavior. - int flags = GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER; - - if (distributedJoins) - flags |= GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS; - - if (qry.isLocal()) - flags |= GridH2QueryRequest.FLAG_IS_LOCAL; - - if (qry.explain()) - flags |= GridH2QueryRequest.FLAG_EXPLAIN; - - if (isReplicatedOnly) - flags |= GridH2QueryRequest.FLAG_REPLICATED; - - if (lazy && mapQrys.size() == 1) - flags |= GridH2QueryRequest.FLAG_LAZY; - GridH2QueryRequest req = new GridH2QueryRequest() .requestId(qryReqId) .topologyVersion(topVer) @@ -778,31 +760,14 @@ public Iterator> query( .partitions(convert(partsMap)) .queries(mapQrys) .parameters(params) - .flags(flags) + .flags(prepareFlags(qry, lazy, mapQrys.size())) .timeout(timeoutMillis) .schemaName(schemaName); if (send(nodes, req, parts == null ? null : new ExplicitPartitionsSpecializer(qryMap), false)) { awaitAllReplies(r, nodes, cancel); - if (r.hasError()) { - if (r.cacheException() != null) { - CacheException err = r.cacheException(); - - if (err.getCause() instanceof IgniteClientDisconnectedException) - throw err; - - if (wasCancelled(err)) - throw new QueryCancelledException(); // Throw correct exception. - - throw new CacheException("Failed to run map query remotely." + err.getMessage(), err); - } else { - retry = true; - - // If remote node asks us to retry then we have outdated full partition map. - h2.awaitForReadyTopologyVersion(r.topVersion()); - } - } + retry = analyseCurrentRun(r); } else // Send failed. retry = true; @@ -876,6 +841,14 @@ public Iterator> query( Throwable disconnectedErr = ((IgniteCheckedException)e).getCause(IgniteClientDisconnectedException.class); + if ( QueryCancelledException.class.isAssignableFrom(e.getClass()) ) + cause = new QueryCancelledException(String.format( + "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s]", + qry.originalSql(), + ctx.localNodeId(), + "Cancelled by client" + )); + if (disconnectedErr != null) cause = disconnectedErr; } @@ -895,6 +868,64 @@ public Iterator> query( } } + /** + * Analyse reduce query run to decide if retry is required + * @param r reduce query run to be analysed + * @return true if retry is required, false otherwise + * @throws IgniteCheckedException in case of reduce query run contains exception record + */ + private boolean analyseCurrentRun(ReduceQueryRun r) throws IgniteCheckedException { + if (r.hasError()) { + if (r.cacheException() != null) { + CacheException err = r.cacheException(); + + if (err.getCause() instanceof IgniteClientDisconnectedException) + throw err; + + Exception cause = wasCancelled(err) || X.hasCause(err, QueryCancelledException.class) + ? new QueryCancelledException(r.rootCause()) + : err; + + throw new CacheException("Failed to run map query remotely." + cause.getMessage(), cause); + } else { + // If remote node asks us to retry then we have outdated full partition map. + h2.awaitForReadyTopologyVersion(r.topVersion()); + + return true; + } + } + return false; + } + + /** + * Builds flag out of parameters + * @param qry query parameter holder + * @param lazy if lazy execution + * @param mapQrysSize number of queries + * @return flag + */ + private int prepareFlags(GridCacheTwoStepQuery qry, boolean lazy, int mapQrysSize) { + // Always enforce join order on map side to have consistent behavior. + int flags = GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER; + + if (qry.distributedJoins()) + flags |= GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS; + + if (qry.isLocal()) + flags |= GridH2QueryRequest.FLAG_IS_LOCAL; + + if (qry.explain()) + flags |= GridH2QueryRequest.FLAG_EXPLAIN; + + if (qry.isReplicatedOnly()) + flags |= GridH2QueryRequest.FLAG_REPLICATED; + + if (lazy && mapQrysSize == 1) + flags |= GridH2QueryRequest.FLAG_LAZY; + + return flags; + } + /** * * @param schemaName Schema name. diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 0fee68c7ea480..42897df265ed3 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -83,16 +83,16 @@ class ReduceQueryRun { * @param o Fail state object. * @param nodeId Node ID. */ - void state(Object o, @Nullable UUID nodeId) { + void state(String msg, Object o, @Nullable UUID nodeId) { assert o != null; assert o instanceof CacheException || o instanceof AffinityTopologyVersion : o.getClass(); if ( o instanceof CacheException ) - state(new State((CacheException)o, null, null, nodeId)); + state(new State((CacheException)o, msg, null, nodeId)); else - state(new State(null, null, (AffinityTopologyVersion)o, nodeId)); + state(new State(null, msg, (AffinityTopologyVersion)o, nodeId)); } /** @@ -126,7 +126,7 @@ private void state(State state){ * @param e Error. */ void disconnected(CacheException e) { - state(e, null); + state("", e, null); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index a0cea727cc6b0..d05a33e3c39c9 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -35,6 +35,7 @@ import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +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.spi.discovery.tcp.TcpDiscoverySpi; @@ -46,7 +47,7 @@ */ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends GridCommonAbstractTest { /** Grids count. */ - private static final int GRIDS_CNT = 3; + private static final int GRIDS_CNT = 4; /** IP finder. */ private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -66,6 +67,12 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr /** */ private static final String QRY_3 = "select a._val from String a"; + /** */ + private static final String CANCELLED_BY_CLIENT = "reason=Cancelled by client"; + + /** */ + private static final String WITH_TIMEOUT_WAS_CANCELLED = "reason=Statement with timeout was cancelled"; + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); @@ -100,82 +107,96 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr /** */ public void testRemoteQueryExecutionTimeout() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, true); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, true, + WITH_TIMEOUT_WAS_CANCELLED); } /** */ - public void testRemoteQueryWithMergeTableTimeout() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, true); + public void testRemoteQueryWithMergeTableTimeout0() throws Exception { + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.MILLISECONDS, true, + WITH_TIMEOUT_WAS_CANCELLED); + } + + /** Query possibly could be executed faster than timeout*/ + public void testRemoteQueryWithMergeTableTimeout1() throws Exception { + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 25, TimeUnit.MILLISECONDS, true, + WITH_TIMEOUT_WAS_CANCELLED); } /** */ public void testRemoteQueryExecutionCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryExecutionCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryExecutionCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryExecutionCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryWithMergeTableCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithMergeTableCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithMergeTableCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithMergeTableCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false, null); } - /** */ + /** Query possibly could be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(2*CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryAlreadyFinishedStop() throws Exception { - testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); + testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); } /** */ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, - boolean timeout) throws Exception { + boolean timeout, String cause) throws Exception { try (Ignite client = startGrid("client")) { IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); @@ -216,18 +237,25 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni try(QueryCursor> ignored = cursor) { cursor.iterator(); - fail(); + + if (!F.isEmpty(cause)) + fail("No exception caught"); } catch (CacheException ex) { - log().error("Got expected exception", ex); + log().error("Got exception", ex); + + log().error( "Cause of exception", ex.getCause()); assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException); - } - // Give some time to clean up. - Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); + assertTrue( "Cause message "+ex.getCause().getMessage(), ex.getCause().getMessage().contains(cause)); + }finally { - checkCleanState(); + // Give some time to clean up. + Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); + + checkCleanState(); + } } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java index fc681a4f33009..4e74ac1e60489 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java @@ -42,6 +42,12 @@ public class IgniteCacheLocalQueryCancelOrTimeoutSelfTest extends GridCommonAbst /** */ private static final String QUERY = "select a._val, b._val from String a, String b"; + /** */ + private static final String CANCELLED_BY_CLIENT = "reason=Cancelled by client"; + + /** */ + private static final String WITH_TIMEOUT_WAS_CANCELLED = "reason=Statement with timeout was cancelled"; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); @@ -93,27 +99,27 @@ private void loadCache(IgniteCache cache) { * Tests cancellation. */ public void testQueryCancel() { - testQuery(false, 1, TimeUnit.SECONDS); + testQuery(false, 1, TimeUnit.SECONDS, CANCELLED_BY_CLIENT); } /** * Tests cancellation with zero timeout. */ public void testQueryCancelZeroTimeout() { - testQuery(false, 1, TimeUnit.MILLISECONDS); + testQuery(false, 1, TimeUnit.MILLISECONDS, CANCELLED_BY_CLIENT); } /** * Tests timeout. */ public void testQueryTimeout() { - testQuery(true, 1, TimeUnit.SECONDS); + testQuery(true, 1, TimeUnit.SECONDS, WITH_TIMEOUT_WAS_CANCELLED); } /** * Tests cancellation. */ - private void testQuery(boolean timeout, int timeoutUnits, TimeUnit timeUnit) { + private void testQuery(boolean timeout, int timeoutUnits, TimeUnit timeUnit, String cause) { Ignite ignite = grid(0); IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); @@ -143,9 +149,16 @@ private void testQuery(boolean timeout, int timeoutUnits, TimeUnit timeUnit) { fail("Expecting timeout"); } catch (Exception e) { + log().error("Got exception", e); + + log().error( "Cause of exception", e.getCause()); + assertTrue("Must throw correct exception", e.getCause() instanceof QueryCancelledException); + + assertTrue( "Cause message "+e.getCause().getMessage(), e.getCause().getMessage().contains(cause)); + } // Test must exit gracefully. } -} \ No newline at end of file +} From ce15a79ded414f3b41bbe8ca3521f5ff4aaa3174 Mon Sep 17 00:00:00 2001 From: zaleslaw Date: Tue, 28 Aug 2018 14:03:37 +0300 Subject: [PATCH 10/95] IGNITE-9393:[ML] KMeans fails on complex data in cache this closes #4628 --- .../ml/clustering/kmeans/KMeansTrainer.java | 60 +++++++---- .../ml/knn/ann/ANNClassificationTrainer.java | 12 ++- .../KNNClassificationModel.java | 9 +- .../ml/math/isolve/lsqr/LSQROnHeap.java | 8 +- .../linear/LinearRegressionSGDTrainer.java | 10 +- .../LogisticRegressionSGDTrainer.java | 12 ++- .../LogRegressionMultiClassTrainer.java | 25 +++-- .../SVMLinearBinaryClassificationTrainer.java | 68 ++++++++++-- ...LinearMultiClassClassificationTrainer.java | 34 +++++- .../ignite/ml/knn/ANNClassificationTest.java | 3 - .../svm/SVMBinaryTrainerIntegrationTest.java | 102 ------------------ .../ignite/ml/svm/SVMBinaryTrainerTest.java | 3 +- .../ml/svm/SVMMultiClassTrainerTest.java | 3 +- .../apache/ignite/ml/svm/SVMTestSuite.java | 1 - 14 files changed, 191 insertions(+), 159 deletions(-) delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerIntegrationTest.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java index c005312cb0069..5b880fcc95ced 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java @@ -65,13 +65,13 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { /** * Trains model based on the specified data. * - * @param datasetBuilder Dataset builder. + * @param datasetBuilder Dataset builder. * @param featureExtractor Feature extractor. - * @param lbExtractor Label extractor. + * @param lbExtractor Label extractor. * @return Model. */ @Override public KMeansModel fit(DatasetBuilder datasetBuilder, - IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { assert datasetBuilder != null; PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( @@ -85,7 +85,14 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder )) { - final int cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> a == null ? b : a); + final int cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> { + if (a == null) + return b == null ? 0 : b; + if (b == null) + return a; + return b; + }); + centers = initClusterCentersRandomly(dataset, k); boolean converged = false; @@ -113,7 +120,8 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { centers[i] = newCentroids[i]; } } - } catch (Exception e) { + } + catch (Exception e) { throw new RuntimeException(e); } return new KMeansModel(centers, distance); @@ -124,15 +132,14 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { * * @param centers Current centers on the current iteration. * @param dataset Dataset. - * @param cols Amount of columns. + * @param cols Amount of columns. * @return Helper data to calculate the new centroids. */ private TotalCostAndCounts calcDataForNewCentroids(Vector[] centers, - Dataset> dataset, int cols) { + Dataset> dataset, int cols) { final Vector[] finalCenters = centers; return dataset.compute(data -> { - TotalCostAndCounts res = new TotalCostAndCounts(); for (int i = 0; i < data.rowSize(); i++) { @@ -147,20 +154,29 @@ private TotalCostAndCounts calcDataForNewCentroids(Vector[] centers, int finalI = i; res.sums.compute(centroidIdx, - (IgniteBiFunction) (ind, v) -> v.plus(data.getRow(finalI).features())); + (IgniteBiFunction)(ind, v) -> { + Vector features = data.getRow(finalI).features(); + return v == null ? features : v.plus(features); + }); res.counts.merge(centroidIdx, 1, - (IgniteBiFunction) (i1, i2) -> i1 + i2); + (IgniteBiFunction)(i1, i2) -> i1 + i2); } return res; - }, (a, b) -> a == null ? b : a.merge(b)); + }, (a, b) -> { + if (a == null) + return b == null ? new TotalCostAndCounts() : b; + if (b == null) + return a; + return a.merge(b); + }); } /** * Find the closest cluster center index and distance to it from a given point. * * @param centers Centers to look in. - * @param pnt Point. + * @param pnt Point. */ private IgniteBiTuple findClosestCentroid(Vector[] centers, LabeledVector pnt) { double bestDistance = Double.POSITIVE_INFINITY; @@ -180,12 +196,11 @@ private IgniteBiTuple findClosestCentroid(Vector[] centers, Lab * K cluster centers are initialized randomly. * * @param dataset The dataset to pick up random centers. - * @param k Amount of clusters. + * @param k Amount of clusters. * @return K cluster centers. */ private Vector[] initClusterCentersRandomly(Dataset> dataset, - int k) { - + int k) { Vector[] initCenters = new DenseVector[k]; // Gets k or less vectors from each partition. @@ -211,12 +226,19 @@ private Vector[] initClusterCentersRandomly(Dataset a == null ? b : Stream.concat(a.stream(), b.stream()).collect(Collectors.toList())); + }, (a, b) -> { + if (a == null) + return b == null ? new ArrayList<>() : b; + if (b == null) + return a; + return Stream.concat(a.stream(), b.stream()).collect(Collectors.toList()); + }); // Shuffle them. Collections.shuffle(rndPnts); @@ -228,7 +250,8 @@ private Vector[] initClusterCentersRandomly(Dataset counts = new ConcurrentHashMap<>(); - /** Count of points closest to the center with a given index. */ ConcurrentHashMap> centroidStat = new ConcurrentHashMap<>(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java index 282be3c15aabd..1c45812908797 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java @@ -149,9 +149,7 @@ private CentroidStat getCentroidStat(DatasetBuilder datasetBuilder, (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder )) { - return dataset.compute(data -> { - CentroidStat res = new CentroidStat(); for (int i = 0; i < data.rowSize(); i++) { @@ -171,7 +169,7 @@ private CentroidStat getCentroidStat(DatasetBuilder datasetBuilder, centroidStat.put(lb, 1); res.centroidStat.put(centroidIdx, centroidStat); } else { - int cnt = centroidStat.containsKey(lb) ? centroidStat.get(lb) : 0; + int cnt = centroidStat.getOrDefault(lb, 0); centroidStat.put(lb, cnt + 1); } @@ -179,7 +177,13 @@ private CentroidStat getCentroidStat(DatasetBuilder datasetBuilder, (IgniteBiFunction) (i1, i2) -> i1 + i2); } return res; - }, (a, b) -> a == null ? b : a.merge(b)); + }, (a, b) -> { + if (a == null) + return b == null ? new CentroidStat() : b; + if (b == null) + return a; + return a.merge(b); + }); } catch (Exception e) { throw new RuntimeException(e); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java index 3404ae80fe939..0b88f8181cf4f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java @@ -17,6 +17,7 @@ package org.apache.ignite.ml.knn.classification; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -79,7 +80,13 @@ protected List findKNearestNeighbors(Vector v) { List neighborsFromPartitions = dataset.compute(data -> { TreeMap> distanceIdxPairs = getDistances(v, data); return Arrays.asList(getKClosestVectors(data, distanceIdxPairs)); - }, (a, b) -> a == null ? b : Stream.concat(a.stream(), b.stream()).collect(Collectors.toList())); + }, (a, b) -> { + if (a == null) + return b == null ? new ArrayList<>() : b; + if (b == null) + return a; + return Stream.concat(a.stream(), b.stream()).collect(Collectors.toList()); + }); LabeledVectorSet neighborsToFilter = buildLabeledDatasetOnListOfVectors(neighborsFromPartitions); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java index e138cf3ff3db9..f75caefab76a4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java @@ -103,7 +103,13 @@ else if (b == null) @Override protected int getColumns() { return dataset.compute( data -> data.getFeatures() == null ? null : data.getFeatures().length / data.getRows(), - (a, b) -> a == null ? b : a + (a, b) -> { + if (a == null) + return b == null ? 0 : b; + if (b == null) + return a; + return b; + } ); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java index 2237c95f1dc12..44f60d1db6a16 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java @@ -82,7 +82,13 @@ public LinearRegressionSGDTrainer(UpdatesStrategy a == null ? b : a); + }, (a, b) -> { + if (a == null) + return b == null ? 0 : b; + if (b == null) + return a; + return b; + }); MLPArchitecture architecture = new MLPArchitecture(cols); architecture = architecture.withAddedLayer(1, true, Activators.LINEAR); @@ -100,7 +106,7 @@ public LinearRegressionSGDTrainer(UpdatesStrategy lbE = (IgniteBiFunction)(k, v) -> new double[]{lbExtractor.apply(k, v)}; + IgniteBiFunction lbE = (IgniteBiFunction)(k, v) -> new double[] {lbExtractor.apply(k, v)}; MultilayerPerceptron mlp = trainer.fit(datasetBuilder, featureExtractor, lbE); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java index 840a18dcd29bc..639627950ac70 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java @@ -64,7 +64,7 @@ public class LogisticRegressionSGDTrainer

extends Single * @param seed Seed for random generator. */ public LogisticRegressionSGDTrainer(UpdatesStrategy updatesStgy, int maxIterations, - int batchSize, int locIterations, long seed) { + int batchSize, int locIterations, long seed) { this.updatesStgy = updatesStgy; this.maxIterations = maxIterations; this.batchSize = batchSize; @@ -82,7 +82,13 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy a == null ? b : a); + }, (a, b) -> { + if (a == null) + return b == null ? 0 : b; + if (b == null) + return a; + return b; + }); MLPArchitecture architecture = new MLPArchitecture(cols); architecture = architecture.withAddedLayer(1, true, Activators.SIGMOID); @@ -100,7 +106,7 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy new double[]{lbExtractor.apply(k, v)}); + MultilayerPerceptron mlp = trainer.fit(datasetBuilder, featureExtractor, (k, v) -> new double[] {lbExtractor.apply(k, v)}); double[] params = mlp.parameters().getStorage().data(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java index 1ed938a4440f0..4885373ae094f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java @@ -61,14 +61,14 @@ public class LogRegressionMultiClassTrainer

/** * Trains model based on the specified data. * - * @param datasetBuilder Dataset builder. + * @param datasetBuilder Dataset builder. * @param featureExtractor Feature extractor. - * @param lbExtractor Label extractor. + * @param lbExtractor Label extractor. * @return Model. */ @Override public LogRegressionMultiClassModel fit(DatasetBuilder datasetBuilder, - IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor) { + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { List classes = extractClassLabels(datasetBuilder, lbExtractor); LogRegressionMultiClassModel multiClsMdl = new LogRegressionMultiClassModel(); @@ -92,7 +92,8 @@ public class LogRegressionMultiClassTrainer

} /** Iterates among dataset and collects class labels. */ - private List extractClassLabels(DatasetBuilder datasetBuilder, IgniteBiFunction lbExtractor) { + private List extractClassLabels(DatasetBuilder datasetBuilder, + IgniteBiFunction lbExtractor) { assert datasetBuilder != null; PartitionDataBuilder partDataBuilder = new LabelPartitionDataBuilderOnHeap<>(lbExtractor); @@ -108,14 +109,22 @@ private List extractClassLabels(DatasetBuilder datasetBuild final double[] lbs = data.getY(); - for (double lb : lbs) locClsLabels.add(lb); + for (double lb : lbs) + locClsLabels.add(lb); return locClsLabels; - }, (a, b) -> a == null ? b : Stream.of(a, b).flatMap(Collection::stream).collect(Collectors.toSet())); + }, (a, b) -> { + if (a == null) + return b == null ? new HashSet<>() : b; + if (b == null) + return a; + return Stream.of(a, b).flatMap(Collection::stream).collect(Collectors.toSet()); + }); res.addAll(clsLabels); - } catch (Exception e) { + } + catch (Exception e) { throw new RuntimeException(e); } return res; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java index 4f11318bd3a27..933a7128731be 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java @@ -17,7 +17,7 @@ package org.apache.ignite.ml.svm; -import java.util.concurrent.ThreadLocalRandom; +import java.util.Random; import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; @@ -47,12 +47,15 @@ public class SVMLinearBinaryClassificationTrainer extends SingleLabelDatasetTrai /** Regularization parameter. */ private double lambda = 0.4; + /** The seed number. */ + private long seed; + /** * Trains model based on the specified data. * - * @param datasetBuilder Dataset builder. + * @param datasetBuilder Dataset builder. * @param featureExtractor Feature extractor. - * @param lbExtractor Label extractor. + * @param lbExtractor Label extractor. * @return Model. */ @Override public SVMLinearBinaryClassificationModel fit(DatasetBuilder datasetBuilder, @@ -67,19 +70,28 @@ public class SVMLinearBinaryClassificationTrainer extends SingleLabelDatasetTrai Vector weights; - try(Dataset> dataset = datasetBuilder.build( + try (Dataset> dataset = datasetBuilder.build( (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder )) { - final int cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> a == null ? b : a); + final int cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> { + if (a == null) + return b == null ? 0 : b; + if (b == null) + return a; + return b; + }); + final int weightVectorSizeWithIntercept = cols + 1; + weights = initializeWeightsWithZeros(weightVectorSizeWithIntercept); for (int i = 0; i < this.getAmountOfIterations(); i++) { Vector deltaWeights = calculateUpdates(weights, dataset); weights = weights.plus(deltaWeights); // creates new vector } - } catch (Exception e) { + } + catch (Exception e) { throw new RuntimeException(e); } return new SVMLinearBinaryClassificationModel(weights.viewPart(1, weights.size() - 1), weights.get(0)); @@ -87,11 +99,12 @@ public class SVMLinearBinaryClassificationTrainer extends SingleLabelDatasetTrai /** */ @NotNull private Vector initializeWeightsWithZeros(int vectorSize) { - return new DenseVector(vectorSize); + return new DenseVector(vectorSize); } /** */ - private Vector calculateUpdates(Vector weights, Dataset> dataset) { + private Vector calculateUpdates(Vector weights, + Dataset> dataset) { return dataset.compute(data -> { Vector copiedWeights = weights.copy(); Vector deltaWeights = initializeWeightsWithZeros(weights.size()); @@ -100,8 +113,10 @@ private Vector calculateUpdates(Vector weights, Dataset a == null ? b : a.plus(b)); + }, (a, b) -> { + if (a == null) + return b == null ? new DenseVector() : b; + if (b == null) + return a; + return a.plus(b); + }); } /** */ private Deltas getDeltas(LabeledVectorSet data, Vector copiedWeights, int amountOfObservation, Vector tmpAlphas, - int randomIdx) { + int randomIdx) { LabeledVector row = (LabeledVector)data.getRow(randomIdx); Double lb = (Double)row.label(); Vector v = makeVectorWithInterceptElement(row); @@ -191,6 +212,7 @@ else if (alpha >= 1.0) /** * Set up the regularization parameter. + * * @param lambda The regularization parameter. Should be more than 0.0. * @return Trainer with new lambda parameter value. */ @@ -202,6 +224,7 @@ public SVMLinearBinaryClassificationTrainer withLambda(double lambda) { /** * Gets the regularization lambda. + * * @return The parameter value. */ public double lambda() { @@ -210,6 +233,7 @@ public double lambda() { /** * Gets the amount of outer iterations of SCDA algorithm. + * * @return The parameter value. */ public int getAmountOfIterations() { @@ -218,6 +242,7 @@ public int getAmountOfIterations() { /** * Set up the amount of outer iterations of SCDA algorithm. + * * @param amountOfIterations The parameter value. * @return Trainer with new amountOfIterations parameter value. */ @@ -228,6 +253,7 @@ public SVMLinearBinaryClassificationTrainer withAmountOfIterations(int amountOfI /** * Gets the amount of local iterations of SCDA algorithm. + * * @return The parameter value. */ public int getAmountOfLocIterations() { @@ -236,6 +262,7 @@ public int getAmountOfLocIterations() { /** * Set up the amount of local iterations of SCDA algorithm. + * * @param amountOfLocIterations The parameter value. * @return Trainer with new amountOfLocIterations parameter value. */ @@ -244,6 +271,25 @@ public SVMLinearBinaryClassificationTrainer withAmountOfLocIterations(int amount return this; } + /** + * Gets the seed number. + * + * @return The parameter value. + */ + public long getSeed() { + return seed; + } + + /** + * Set up the seed. + * + * @param seed The parameter value. + * @return Model with new seed parameter value. + */ + public SVMLinearBinaryClassificationTrainer withSeed(long seed) { + this.seed = seed; + return this; + } } /** This is a helper class to handle pair results which are returned from the calculation method. */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java index 7069c4d0d5765..4b7cc95ebe1c8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java @@ -51,6 +51,9 @@ public class SVMLinearMultiClassClassificationTrainer /** Regularization parameter. */ private double lambda = 0.2; + /** The seed number. */ + private long seed; + /** * Trains model based on the specified data. * @@ -70,7 +73,8 @@ public class SVMLinearMultiClassClassificationTrainer SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer() .withAmountOfIterations(this.amountOfIterations()) .withAmountOfLocIterations(this.amountOfLocIterations()) - .withLambda(this.lambda()); + .withLambda(this.lambda()) + .withSeed(this.seed); IgniteBiFunction lbTransformer = (k, v) -> { Double lb = lbExtractor.apply(k, v); @@ -106,7 +110,13 @@ private List extractClassLabels(DatasetBuilder datasetBuild for (double lb : lbs) locClsLabels.add(lb); return locClsLabels; - }, (a, b) -> a == null ? b : Stream.of(a, b).flatMap(Collection::stream).collect(Collectors.toSet())); + }, (a, b) -> { + if (a == null) + return b == null ? new HashSet<>() : b; + if (b == null) + return a; + return Stream.of(a, b).flatMap(Collection::stream).collect(Collectors.toSet()); + }); res.addAll(clsLabels); @@ -176,4 +186,24 @@ public SVMLinearMultiClassClassificationTrainer withAmountOfLocIterations(int a this.amountOfLocIterations = amountOfLocIterations; return this; } + + /** + * Gets the seed number. + * + * @return The parameter value. + */ + public long getSeed() { + return seed; + } + + /** + * Set up the seed. + * + * @param seed The parameter value. + * @return Model with new seed parameter value. + */ + public SVMLinearMultiClassClassificationTrainer withSeed(long seed) { + this.seed = seed; + return this; + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java index aed638788e091..7289b1dfaf1f7 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java @@ -62,9 +62,6 @@ public void testBinaryClassification() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(NNStrategy.SIMPLE); - TestUtils.assertEquals(0, mdl.apply(VectorUtils.of(550, 550)), PRECISION); - TestUtils.assertEquals(1, mdl.apply(VectorUtils.of(-550, -550)), PRECISION); - Assert.assertNotNull(((ANNClassificationModel) mdl).getCandidates()); Assert.assertTrue(mdl.toString().contains(NNStrategy.SIMPLE.name())); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerIntegrationTest.java deleted file mode 100644 index d227de7e329e0..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerIntegrationTest.java +++ /dev/null @@ -1,102 +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.ml.svm; - -import java.util.Arrays; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.TestUtils; -import org.apache.ignite.ml.math.primitives.vector.VectorUtils; -import org.apache.ignite.ml.math.primitives.vector.impl.DenseVector; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Tests for {@link SVMLinearBinaryClassificationTrainer} that require to start the whole Ignite infrastructure. - */ -public class SVMBinaryTrainerIntegrationTest extends GridCommonAbstractTest { - /** Fixed size of Dataset. */ - private static final int AMOUNT_OF_OBSERVATIONS = 1000; - - /** Fixed size of columns in Dataset. */ - private static final int AMOUNT_OF_FEATURES = 2; - - /** Precision in test checks. */ - private static final double PRECISION = 1e-2; - - /** Number of nodes in grid */ - private static final int NODE_COUNT = 3; - - /** Ignite instance. */ - private Ignite ignite; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() { - stopAllGrids(); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - /* Grid instance. */ - ignite = grid(NODE_COUNT); - ignite.configuration().setPeerClassLoadingEnabled(true); - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - } - - /** - * Test trainer on classification model y = x. - */ - public void testTrainWithTheLinearlySeparableCase() { - IgniteCache data = ignite.getOrCreateCache(UUID.randomUUID().toString()); - - ThreadLocalRandom rndX = ThreadLocalRandom.current(); - ThreadLocalRandom rndY = ThreadLocalRandom.current(); - - for (int i = 0; i < AMOUNT_OF_OBSERVATIONS; i++) { - double x = rndX.nextDouble(-1000, 1000); - double y = rndY.nextDouble(-1000, 1000); - double[] vec = new double[AMOUNT_OF_FEATURES + 1]; - vec[0] = y - x > 0 ? 1 : -1; // assign label. - vec[1] = x; - vec[2] = y; - data.put(i, vec); - } - - SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer(); - - SVMLinearBinaryClassificationModel mdl = trainer.fit( - ignite, - data, - (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), - (k, v) -> v[0] - ); - - TestUtils.assertEquals(-1, mdl.apply(new DenseVector(new double[]{100, 10})), PRECISION); - TestUtils.assertEquals(1, mdl.apply(new DenseVector(new double[]{10, 100})), PRECISION); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java index b7721774d4098..5630beea5f370 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java @@ -39,7 +39,8 @@ public void testTrainWithTheLinearlySeparableCase() { for (int i = 0; i < twoLinearlySeparableClasses.length; i++) cacheMock.put(i, twoLinearlySeparableClasses[i]); - SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer(); + SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer() + .withSeed(1234L); SVMLinearBinaryClassificationModel mdl = trainer.fit( cacheMock, diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java index f2328f8752a4b..7ea28c2493f26 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java @@ -42,7 +42,8 @@ public void testTrainWithTheLinearlySeparableCase() { SVMLinearMultiClassClassificationTrainer trainer = new SVMLinearMultiClassClassificationTrainer() .withLambda(0.3) .withAmountOfLocIterations(10) - .withAmountOfIterations(20); + .withAmountOfIterations(20) + .withSeed(1234L); SVMLinearMultiClassClassificationModel mdl = trainer.fit( cacheMock, diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMTestSuite.java index 822ad184ea929..df7263f9d47f7 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMTestSuite.java @@ -28,7 +28,6 @@ SVMModelTest.class, SVMBinaryTrainerTest.class, SVMMultiClassTrainerTest.class, - SVMBinaryTrainerIntegrationTest.class }) public class SVMTestSuite { // No-op. From 435184a203598740ace2f0b53cf136dc3443e168 Mon Sep 17 00:00:00 2001 From: Roman Guseinov Date: Tue, 28 Aug 2018 14:41:09 +0300 Subject: [PATCH 11/95] IGNITE-9367: Fixed crash in ODBC on executing query with closed connection This closes #4621 --- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../cpp/odbc-test/src/connection_test.cpp | 133 ++++++++++++++++++ modules/platforms/cpp/odbc/src/connection.cpp | 3 + 5 files changed, 141 insertions(+) create mode 100644 modules/platforms/cpp/odbc-test/src/connection_test.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 87e3c893398c8..f41240365b30d 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -58,6 +58,7 @@ ignite_odbc_tests_SOURCES = \ src/parser_test.cpp \ src/cursor_test.cpp \ src/connection_info_test.cpp \ + src/connection_test.cpp \ src/application_data_buffer_test.cpp \ src/column_test.cpp \ src/configuration_test.cpp \ diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index e4af9604f756c..87b655936c1f5 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -176,6 +176,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index e2749caf2c61d..3d2fcc6dfd13c 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -25,6 +25,9 @@ Code + + Code + Code diff --git a/modules/platforms/cpp/odbc-test/src/connection_test.cpp b/modules/platforms/cpp/odbc-test/src/connection_test.cpp new file mode 100644 index 0000000000000..5badda65a55d5 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/connection_test.cpp @@ -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. + */ + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignite.h" +#include "ignite/ignition.h" + +#include "test_type.h" +#include "test_utils.h" +#include "odbc_test_suite.h" + +using namespace ignite; +using namespace ignite::common; +using namespace ignite_test; + +using namespace boost::unit_test; + +/** + * Test setup fixture. + */ +struct ConnectionTestSuiteFixture: odbc::OdbcTestSuite +{ + /** + * Constructor. + */ + ConnectionTestSuiteFixture() : + OdbcTestSuite() + { + StartNode(); + } + + /** + * Start a node. + */ + void StartNode() + { + StartTestNode("queries-test.xml", "NodeMain"); + } + + /** + * Execute the query and return an error code. + */ + std::string ExecQueryAndReturnError() + { + SQLCHAR selectReq[] = "select count(*) from TestType"; + + SQLRETURN ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq)); + + std::string err; + + if (!SQL_SUCCEEDED(ret)) + err = ExtractErrorCode(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + return err; + } + + /** + * Extract code from ODBC error message. + */ + std::string ExtractErrorCode(std::string err) + { + std::string code; + + int idx = err.find(':'); + + if ((idx != std::string::npos) && (idx > 0)) + code = err.substr(0, idx); + + return code; + } + + /** + * Destructor. + */ + ~ConnectionTestSuiteFixture() + { + // No-op. + } +}; + +BOOST_FIXTURE_TEST_SUITE(ConnectionTestSuite, ConnectionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestConnectionRestore) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + + // Check that query was successfully executed. + BOOST_CHECK_EQUAL(ExecQueryAndReturnError(), ""); + + // Stop node. + Ignition::StopAll(true); + + // Query execution should throw ODBC error. + BOOST_CHECK_EQUAL(ExecQueryAndReturnError(), "08S01"); + + // Reusing a closed connection should not crash an application. + BOOST_CHECK_EQUAL(ExecQueryAndReturnError(), "08001"); + + StartNode(); + + // Check that connection was restored. + BOOST_CHECK_EQUAL(ExecQueryAndReturnError(), ""); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index 4a28bbf4dd5f1..894efb7b20999 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -617,6 +617,9 @@ namespace ignite CollectAddresses(config, addrs); + if (socket.get() == 0) + socket.reset(new system::TcpSocketClient()); + bool connected = false; while (!addrs.empty() && !connected) From 82e55e927260327996495360b5ae7019a1c958a6 Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Tue, 28 Aug 2018 17:46:29 +0300 Subject: [PATCH 12/95] IGNITE-9389 Avoid deadlock on cache#close(). - Fixes #4626. Signed-off-by: Alexey Goncharuk --- .../processors/cache/GridCacheProcessor.java | 72 +++++++++---------- .../cache/CacheStopAndDestroySelfTest.java | 68 ++++++++++++++++++ 2 files changed, 100 insertions(+), 40 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 272aad40d90bc..eb64d9c1d6628 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -2182,50 +2182,31 @@ void initCacheProxies(AffinityTopologyVersion startTopVer, @Nullable Throwable e Set closeCaches(Set cachesToClose, boolean retClientCaches) { Set ids = null; - boolean locked = false; + for (String cacheName : cachesToClose) { + blockGateway(cacheName, false, false); - try { - for (String cacheName : cachesToClose) { - blockGateway(cacheName, false, false); - - GridCacheContext ctx = sharedCtx.cacheContext(CU.cacheId(cacheName)); - - if (ctx == null) - continue; - - if (retClientCaches && !ctx.affinityNode()) { - if (ids == null) - ids = U.newHashSet(cachesToClose.size()); - - ids.add(ctx.cacheId()); - } + GridCacheContext ctx = sharedCtx.cacheContext(CU.cacheId(cacheName)); - if (!ctx.affinityNode() && !locked) { - // Do not close client cache while requests processing is in progress. - sharedCtx.io().writeLock(); - - locked = true; - } + if (ctx == null) + continue; - if (!ctx.affinityNode() && ctx.transactional()) - sharedCtx.tm().rollbackTransactionsForCache(ctx.cacheId()); + if (retClientCaches && !ctx.affinityNode()) { + if (ids == null) + ids = U.newHashSet(cachesToClose.size()); - closeCache(ctx, false); + ids.add(ctx.cacheId()); } - return ids; - } - finally { - if (locked) - sharedCtx.io().writeUnlock(); + closeCache(ctx); } + + return ids; } /** * @param cctx Cache context. - * @param destroy Destroy flag. */ - private void closeCache(GridCacheContext cctx, boolean destroy) { + private void closeCache(GridCacheContext cctx) { if (cctx.affinityNode()) { GridCacheAdapter cache = caches.get(cctx.name()); @@ -2234,21 +2215,32 @@ private void closeCache(GridCacheContext cctx, boolean destroy) { jCacheProxies.put(cctx.name(), new IgniteCacheProxyImpl(cache.context(), cache, false)); } else { - jCacheProxies.remove(cctx.name()); - cctx.gate().onStopped(); - sharedCtx.database().checkpointReadLock(); + // Do not close client cache while requests processing is in progress. + sharedCtx.io().writeLock(); try { - prepareCacheStop(cctx.name(), destroy); + if (!cctx.affinityNode() && cctx.transactional()) + sharedCtx.tm().rollbackTransactionsForCache(cctx.cacheId()); + + jCacheProxies.remove(cctx.name()); + + sharedCtx.database().checkpointReadLock(); + + try { + prepareCacheStop(cctx.name(), false); + } + finally { + sharedCtx.database().checkpointReadUnlock(); + } + + if (!cctx.group().hasCaches()) + stopCacheGroup(cctx.group().groupId()); } finally { - sharedCtx.database().checkpointReadUnlock(); + sharedCtx.io().writeUnlock(); } - - if (!cctx.group().hasCaches()) - stopCacheGroup(cctx.group().groupId()); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java index 5eb8292a3e43f..6239b525d9eb4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java @@ -18,11 +18,15 @@ package org.apache.ignite.internal.processors.cache; import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import javax.cache.Cache; import javax.cache.CacheManager; import javax.cache.Caching; import javax.cache.configuration.MutableConfiguration; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; @@ -781,6 +785,70 @@ public void testTckStyleCreateDestroyClose() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testConcurrentUseAndCloseFromClient() throws Exception { + testConcurrentUseAndClose(true); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentUseAndCloseFromServer() throws Exception { + testConcurrentUseAndClose(false); + } + + /** + * @param isClient Should client or server be used during the test. + * @throws Exception If failed. + */ + private void testConcurrentUseAndClose(boolean isClient) throws Exception { + int threads = 8; + int keys = 1000; + int iterations = 20; + + startGrid(0); + + IgniteConfiguration igniteCfg = getConfiguration(getTestIgniteInstanceName(1)); + igniteCfg.setClientMode(isClient); + Ignite ignite = startGrid(optimize(igniteCfg)); + + ExecutorService execSrvc = Executors.newFixedThreadPool(threads); + + for (int i = 0; i < threads; i++) { + execSrvc.execute(() -> { + while (!Thread.interrupted()) { + try { + IgniteCache cache = ignite.getOrCreateCache("cache"); + + ThreadLocalRandom random = ThreadLocalRandom.current(); + int key = random.nextInt(keys); + + if (random.nextBoolean()) + cache.put(key, Integer.toString(key)); + else + cache.get(key); + } + catch (Exception ignore) { + } + } + }); + } + + for (int i = 0; i < iterations; i++) { + System.out.println("Iteration #" + (i + 1)); + + IgniteCache cache = ignite.getOrCreateCache("cache"); + + cache.close(); + + Thread.sleep(100); + } + + execSrvc.shutdownNow(); + } + /** * @param cache Cache. * @throws Exception If failed. From a4dedb746a5997d8f318b7ee059825e7b7101687 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Tue, 28 Aug 2018 18:18:26 +0300 Subject: [PATCH 13/95] IGNITE-9401 Fixed race in tx rollback test - Fixes #4633. Signed-off-by: Alexey Goncharuk --- .../processors/cache/transactions/TxRollbackAsyncTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java index 646c3f01fb316..7968be31b9146 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java @@ -27,6 +27,7 @@ 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.AtomicIntegerArray; import java.util.concurrent.atomic.AtomicReference; @@ -900,7 +901,7 @@ public void testRollbackOnTopologyLockPessimistic() throws Exception { txLatch.countDown(); - U.awaitQuiet(commitLatch); + assertTrue(U.await(commitLatch, 10, TimeUnit.SECONDS)); tx.commit(); @@ -931,6 +932,8 @@ public void testRollbackOnTopologyLockPessimistic() throws Exception { crd.cache(CACHE_NAME).put(keys.get(0), 0); + assertTrue(U.await(commitLatch, 10, TimeUnit.SECONDS)); + tx.commit(); fail(); From b8b2e7098a675a4f37264579be1dfd2c89f3316c Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Wed, 29 Aug 2018 15:32:59 +0300 Subject: [PATCH 14/95] IGNITE-6879 Support Spring Data 2.0 - Added package description to parent/pom.xml --- parent/pom.xml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/parent/pom.xml b/parent/pom.xml index 6b18bbbf160a0..76e39b79e29fa 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -468,7 +468,11 @@ org.apache.ignite.springdata.repository* - RocketMQ integration + SpringData 2.0 integration + org.apache.ignite.springdata20.repository* + + + RocketMQ integration org.apache.ignite.stream.rocketmq* From c1f5a85c61ad6086b8b180b3169fed30f1155841 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 30 Aug 2018 11:35:53 +0300 Subject: [PATCH 15/95] IGNITE-9326 Fixed deferred serialization error handling for EntryProcessor result - Fixes #4588. Signed-off-by: Alexey Goncharuk --- .../cache/CacheInvokeDirectResult.java | 21 ++++++++++++++----- .../processors/cache/GridCacheReturn.java | 17 ++++++++++++++- .../dht/atomic/GridDhtAtomicCache.java | 4 ++++ .../IgniteCacheFailedUpdateResponseTest.java | 2 -- 4 files changed, 36 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java index 89a0a0ff7c000..3b463afe8a25e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java @@ -140,16 +140,27 @@ public void prepareMarshal(GridCacheContext ctx) throws IgniteCheckedException { } } - if (unprepareRes != null) { - res = ctx.toCacheObject(unprepareRes); - - unprepareRes = null; - } + assert unprepareRes == null : "marshalResult() was not called for the result: " + this; if (res != null) res.prepareMarshal(ctx.cacheObjectContext()); } + /** + * Converts the entry processor unprepared result to a cache object instance. + * + * @param ctx Cache context. + */ + public void marshalResult(GridCacheContext ctx) { + try { + if (unprepareRes != null) + res = ctx.toCacheObject(unprepareRes); + } + finally { + unprepareRes = null; + } + } + /** * @param ctx Cache context. * @param ldr Class loader. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java index 530f5b6581deb..2ae0f7c09eacc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java @@ -269,7 +269,10 @@ else if (err instanceof UnregisteredBinaryTypeException) invokeResCol = new ArrayList<>(); CacheInvokeDirectResult res0 = err == null ? - CacheInvokeDirectResult.lazyResult(key, res) : new CacheInvokeDirectResult(key, err); + cctx.transactional() ? + new CacheInvokeDirectResult(key, cctx.toCacheObject(res)) : + CacheInvokeDirectResult.lazyResult(key, res) : + new CacheInvokeDirectResult(key, err); invokeResCol.add(res0); } @@ -307,6 +310,18 @@ public synchronized void mergeEntryProcessResults(GridCacheReturn other) { resMap.putAll((Map)other.v); } + /** + * Converts entry processor invokation results to cache object instances. + * + * @param ctx Cache context. + */ + public void marshalResult(GridCacheContext ctx) { + if (invokeRes && invokeResCol != null) { + for (CacheInvokeDirectResult directRes : invokeResCol) + directRes.marshalResult(ctx); + } + } + /** * @param ctx Cache context. * @throws IgniteCheckedException If failed. 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 2e77b5243b6e2..112b91320f39e 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 @@ -1787,6 +1787,10 @@ private void updateAllAsyncInternal0( top.readUnlock(); } + // This call will convert entry processor invocation results to cache object instances. + // Must be done outside topology read lock to avoid deadlocks. + res.returnValue().marshalResult(ctx); + break; } catch (UnregisteredClassException ex) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheFailedUpdateResponseTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheFailedUpdateResponseTest.java index bef33e8649ce5..ebcff7c43898e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheFailedUpdateResponseTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheFailedUpdateResponseTest.java @@ -104,8 +104,6 @@ public class IgniteCacheFailedUpdateResponseTest extends GridCommonAbstractTest * @throws Exception If failed. */ public void testInvokeAtomic() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-9326"); - testInvoke(atomicCache); testInvokeAll(atomicCache); } From ee2c5a7b7c7f0a0684fdaa063c759057c4bfa6ba Mon Sep 17 00:00:00 2001 From: zaleslaw Date: Thu, 30 Aug 2018 11:56:39 +0300 Subject: [PATCH 16/95] IGNITE-9421: ML Examples: LogisticRegressionSGDTrainerExample example result not correct this closes #4646 --- .../logistic/binary/LogisticRegressionSGDTrainerExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java index 99b56772665c2..7e1c773441d72 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java @@ -75,7 +75,7 @@ public static void main(String[] args) throws InterruptedException { dataCache, (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), (k, v) -> v[0] - ).withRawLabels(true); + ); System.out.println(">>> Logistic regression model: " + mdl); From f858474292805cf93a30c01676f519790402df1c Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 30 Aug 2018 12:39:22 +0300 Subject: [PATCH 17/95] IGNITE-9429 Fixed flaky GridCacheReplicatedDataStructuresFailoverSelfTest --- ...heAbstractDataStructuresFailoverSelfTest.java | 16 +++++++++------- ...ReplicatedDataStructuresFailoverSelfTest.java | 8 ++++---- 2 files changed, 13 insertions(+), 11 deletions(-) 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 9da1161b850f5..797e90f6a0d4f 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 @@ -118,11 +118,6 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig // No-op } - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - // No-op - } - /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { startGridsMultiThreaded(gridCount()); @@ -1362,7 +1357,7 @@ IgniteInternalFuture startChangingTopology(final IgniteClosure cb) if (failed.get()) return; - int idx = nodeIdx.getAndIncrement(); + int idx = nodeIdx.incrementAndGet(); Thread.currentThread().setName("thread-" + getTestIgniteInstanceName(idx)); @@ -1373,8 +1368,15 @@ IgniteInternalFuture startChangingTopology(final IgniteClosure cb) cb.apply(g); } + catch (IgniteException e) { + if (!X.hasCause(e, NodeStoppingException.class) && + !X.hasCause(e, IllegalStateException.class)) + throw e; + + // OK for this test. + } finally { - if(circular) + if (circular) stopGrid(G.allGrids().get(0).configuration().getIgniteInstanceName()); else stopGrid(idx); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java index a2287187e0842..cb6e77e8a2e96 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java @@ -40,16 +40,16 @@ public class GridCacheReplicatedDataStructuresFailoverSelfTest } /** - * @throws Exception If failed. + * */ - public void testFairReentrantLockConstantMultipleTopologyChangeNonFailoverSafe() throws Exception { + @Override public void testFairReentrantLockConstantMultipleTopologyChangeNonFailoverSafe() { fail("https://issues.apache.org/jira/browse/IGNITE-6454"); } /** - * @throws Exception If failed. + * */ - public void testReentrantLockConstantMultipleTopologyChangeNonFailoverSafe() throws Exception { + @Override public void testReentrantLockConstantMultipleTopologyChangeNonFailoverSafe() { fail("https://issues.apache.org/jira/browse/IGNITE-6454"); } } From 88d16494516ab66af3849aae8e3aa6faf7f32efb Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 30 Aug 2018 18:11:21 +0700 Subject: [PATCH 18/95] IGNITE-9370 Fixed execution of REST commands in demo mode. --- modules/web-console/backend/app/browsersHandler.js | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/modules/web-console/backend/app/browsersHandler.js b/modules/web-console/backend/app/browsersHandler.js index 5aade60fac3e2..3ae91da7223fa 100644 --- a/modules/web-console/backend/app/browsersHandler.js +++ b/modules/web-console/backend/app/browsersHandler.js @@ -205,10 +205,11 @@ module.exports = { nodeListeners(sock) { // Return command result from grid to browser. sock.on('node:rest', ({clusterId, params, credentials} = {}, cb) => { - if (_.isNil(clusterId) || _.isNil(params)) + const demo = sock.request._query.IgniteDemoMode === 'true'; + + if ((_.isNil(clusterId) && !demo) || _.isNil(params)) return cb('Invalid format of message: "node:rest"'); - const demo = sock.request._query.IgniteDemoMode === 'true'; const token = sock.request.user.token; const agent = this._agentHnd.agent(token, demo, clusterId); @@ -237,10 +238,11 @@ module.exports = { // Return command result from grid to browser. sock.on('node:visor', ({clusterId, params, credentials} = {}, cb) => { - if (_.isNil(clusterId) || _.isNil(params)) + const demo = sock.request._query.IgniteDemoMode === 'true'; + + if ((_.isNil(clusterId) && !demo) || _.isNil(params)) return cb('Invalid format of message: "node:visor"'); - const demo = sock.request._query.IgniteDemoMode === 'true'; const token = sock.request.user.token; const {taskId, nids, args = []} = params; From 23b33b617356704370f6b15b05b41f3edc5b2bd9 Mon Sep 17 00:00:00 2001 From: Igor Seliverstov Date: Thu, 30 Aug 2018 14:52:49 +0300 Subject: [PATCH 19/95] IGNITE-4191: MVCC and transactional SQL support. Joint multi-man-years efforts of Semen Boikov, Igor Seliverstov, Alexander Paschenko, Igor Sapego, Sergey Kalashnikov, Roman Kondakov, Pavel Kuznetsov, Ivan Pavlukhin, Andrey Mashenkov, and many others. Special thanks for design ideas and review to Alexey Goncharuk and Sergi Vladykin. This closes #3220. --- .../jmh/tree/BPlusTreeBenchmark.java | 2 +- .../internal/jdbc2/JdbcStreamingSelfTest.java | 3 +- .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 14 + .../jdbc/thin/JdbcThinAbstractSelfTest.java | 4 +- ...JdbcThinConnectionMvccEnabledSelfTest.java | 378 ++ .../jdbc/thin/JdbcThinConnectionSelfTest.java | 188 +- .../JdbcThinStreamingAbstractSelfTest.java | 19 +- .../JdbcThinStreamingNotOrderedSelfTest.java | 2 +- .../JdbcThinStreamingOrderedSelfTest.java | 2 +- ...inTransactionsAbstractComplexSelfTest.java | 1058 +++++ ...ctionsClientAutoCommitComplexSelfTest.java | 33 + ...ionsClientNoAutoCommitComplexSelfTest.java | 33 + .../thin/JdbcThinTransactionsSelfTest.java | 447 ++ ...ctionsServerAutoCommitComplexSelfTest.java | 33 + ...ionsServerNoAutoCommitComplexSelfTest.java | 33 + .../ignite/codegen/MessageCodeGenerator.java | 6 +- .../configuration/CacheConfiguration.java | 1 + .../configuration/IgniteConfiguration.java | 84 +- .../apache/ignite/internal/GridComponent.java | 5 +- .../ignite/internal/GridKernalContext.java | 6 + .../internal/GridKernalContextImpl.java | 14 +- .../org/apache/ignite/internal/GridTopic.java | 5 +- .../apache/ignite/internal/IgniteKernal.java | 22 +- .../ignite/internal/IgniteNodeAttributes.java | 3 + .../apache/ignite/internal/IgnitionEx.java | 8 +- .../jdbc/thin/ConnectionProperties.java | 10 + .../jdbc/thin/ConnectionPropertiesImpl.java | 37 +- .../jdbc/thin/JdbcThinConnection.java | 33 +- .../jdbc/thin/JdbcThinDatabaseMetadata.java | 9 +- .../internal/jdbc/thin/JdbcThinStatement.java | 7 +- .../internal/jdbc/thin/JdbcThinTcpIo.java | 10 +- .../managers/communication/GridIoManager.java | 16 + .../communication/GridIoMessageFactory.java | 134 +- .../managers/discovery/DiscoCache.java | 17 +- .../discovery/GridDiscoveryManager.java | 21 + .../pagemem/store/IgnitePageStoreManager.java | 13 + .../pagemem/wal/record/WALRecord.java | 11 +- .../delta/DataPageMvccMarkUpdatedRecord.java | 103 + ...ataPageMvccUpdateNewTxStateHintRecord.java | 79 + .../DataPageMvccUpdateTxStateHintRecord.java | 79 + .../affinity/AffinityAssignment.java | 9 +- .../affinity/GridAffinityAssignment.java | 15 +- .../affinity/GridAffinityAssignmentCache.java | 21 +- .../affinity/GridAffinityProcessor.java | 2 +- .../affinity/GridAffinityUtils.java | 2 +- .../affinity/HistoryAffinityAssignment.java | 12 +- .../cache/CacheAffinitySharedManager.java | 33 +- .../cache/CacheEntryInfoCollection.java | 12 + .../processors/cache/CacheGroupContext.java | 34 + .../cache/CacheOperationContext.java | 2 +- .../processors/cache/ClusterCachesInfo.java | 17 +- .../processors/cache/ExchangeContext.java | 43 +- .../cache/GatewayProtectedCacheProxy.java | 3 + .../processors/cache/GridCacheAdapter.java | 65 +- .../cache/GridCacheAffinityManager.java | 5 + .../processors/cache/GridCacheContext.java | 7 + .../processors/cache/GridCacheEntryEx.java | 135 +- .../processors/cache/GridCacheEntryInfo.java | 6 +- .../processors/cache/GridCacheMapEntry.java | 1149 +++++- .../cache/GridCacheMvccEntryInfo.java | 257 ++ .../processors/cache/GridCacheOperation.java | 2 +- .../GridCachePartitionExchangeManager.java | 76 +- .../processors/cache/GridCacheProcessor.java | 28 +- .../cache/GridCacheSharedContext.java | 11 +- .../cache/GridCacheUpdateTxResult.java | 98 +- .../processors/cache/GridCacheUtils.java | 113 +- .../cache/IgniteCacheOffheapManager.java | 481 ++- .../cache/IgniteCacheOffheapManagerImpl.java | 1679 +++++++- .../cache/IgniteCacheProxyImpl.java | 4 + .../distributed/GridCacheTxFinishSync.java | 2 +- .../GridCacheTxRecoveryFuture.java | 11 + .../distributed/GridDistributedTxMapping.java | 17 + .../GridDistributedTxPrepareRequest.java | 22 + .../GridDistributedTxRemoteAdapter.java | 75 +- .../distributed/dht/CompoundLockFuture.java | 112 + .../cache/distributed/dht/DhtLockFuture.java | 30 + .../cache/distributed/dht/ExceptionAware.java | 30 + .../dht/GridClientPartitionTopology.java | 7 + .../distributed/dht/GridDhtCacheAdapter.java | 62 +- .../distributed/dht/GridDhtCacheEntry.java | 10 +- .../distributed/dht/GridDhtGetFuture.java | 15 +- .../dht/GridDhtGetSingleFuture.java | 15 +- .../dht/GridDhtLocalPartition.java | 23 + .../distributed/dht/GridDhtLockFuture.java | 15 +- .../dht/GridDhtPartitionTopology.java | 4 + .../dht/GridDhtPartitionTopologyImpl.java | 11 + .../dht/GridDhtPartitionsStateValidator.java | 11 +- .../GridDhtPartitionsUpdateCountersMap.java | 119 + .../dht/GridDhtTransactionalCacheAdapter.java | 465 ++- .../dht/GridDhtTxAbstractEnlistFuture.java | 1139 +++++ .../dht/GridDhtTxFinishFuture.java | 111 +- .../dht/GridDhtTxFinishRequest.java | 77 +- .../cache/distributed/dht/GridDhtTxLocal.java | 11 +- .../dht/GridDhtTxLocalAdapter.java | 45 +- .../dht/GridDhtTxPrepareFuture.java | 106 +- .../dht/GridDhtTxPrepareRequest.java | 67 +- .../dht/GridDhtTxQueryEnlistFuture.java | 140 + .../dht/GridDhtTxQueryEnlistRequest.java | 404 ++ .../dht/GridDhtTxQueryEnlistResponse.java | 205 + .../dht/GridDhtTxQueryFirstEnlistRequest.java | 370 ++ .../GridDhtTxQueryResultsEnlistFuture.java | 130 + .../distributed/dht/GridDhtTxRemote.java | 144 + .../dht/GridPartitionedGetFuture.java | 90 +- .../dht/GridPartitionedSingleGetFuture.java | 22 +- .../dht/NearTxQueryEnlistResultHandler.java | 169 + .../dht/atomic/GridDhtAtomicCache.java | 21 +- .../dht/colocated/GridDhtColocatedCache.java | 30 +- .../dht/preloader/GridDhtForceKeysFuture.java | 7 + .../preloader/GridDhtPartitionDemander.java | 7 + .../preloader/GridDhtPartitionSupplier.java | 38 +- .../GridDhtPartitionsExchangeFuture.java | 48 +- .../GridDhtPartitionsSingleMessage.java | 76 +- .../near/AckCoordinatorOnRollback.java | 54 + .../distributed/near/GridNearGetFuture.java | 18 +- .../distributed/near/GridNearGetRequest.java | 49 +- ...OptimisticSerializableTxPrepareFuture.java | 75 +- .../GridNearOptimisticTxPrepareFuture.java | 68 +- ...dNearOptimisticTxPrepareFutureAdapter.java | 133 +- .../GridNearPessimisticTxPrepareFuture.java | 194 +- .../near/GridNearSingleGetRequest.java | 41 +- .../near/GridNearTxAbstractEnlistFuture.java | 499 +++ .../near/GridNearTxFastFinishFuture.java | 13 +- .../near/GridNearTxFinishAndAckFuture.java | 144 + .../near/GridNearTxFinishFuture.java | 66 +- .../near/GridNearTxFinishRequest.java | 30 +- .../distributed/near/GridNearTxLocal.java | 407 +- .../near/GridNearTxPrepareFutureAdapter.java | 12 +- .../near/GridNearTxPrepareRequest.java | 17 + .../near/GridNearTxPrepareResponse.java | 50 +- .../near/GridNearTxQueryEnlistFuture.java | 382 ++ .../near/GridNearTxQueryEnlistRequest.java | 616 +++ .../near/GridNearTxQueryEnlistResponse.java | 298 ++ .../GridNearTxQueryResultsEnlistFuture.java | 670 +++ .../GridNearTxQueryResultsEnlistRequest.java | 578 +++ .../GridNearTxQueryResultsEnlistResponse.java | 202 + .../distributed/near/GridNearTxRemote.java | 9 +- .../near/GridNearTxSelectForUpdateFuture.java | 462 +++ .../distributed/near/NearTxFinishFuture.java | 21 +- .../near/TxTopologyVersionFuture.java | 155 + .../cache/dr/GridCacheDrManager.java | 31 + .../cache/dr/GridOsCacheDrManager.java | 13 + .../local/atomic/GridLocalAtomicCache.java | 15 +- .../cache/mvcc/MvccCoordinator.java | 104 + .../cache/mvcc/MvccDiscoveryData.java | 52 + .../cache/mvcc/MvccEmptyLongList.java | 53 + .../processors/cache/mvcc/MvccFuture.java | 66 + .../processors/cache/mvcc/MvccLongList.java | 29 + .../mvcc/MvccPreviousCoordinatorQueries.java | 222 + .../processors/cache/mvcc/MvccProcessor.java | 250 ++ .../cache/mvcc/MvccProcessorImpl.java | 2187 ++++++++++ .../cache/mvcc/MvccQueryTracker.java | 102 + .../cache/mvcc/MvccQueryTrackerImpl.java | 348 ++ .../processors/cache/mvcc/MvccSnapshot.java | 48 + .../cache/mvcc/MvccSnapshotFuture.java | 43 + .../mvcc/MvccSnapshotResponseListener.java | 35 + .../cache/mvcc/MvccSnapshotWithoutTxs.java | 204 + .../cache/mvcc/MvccUpdateVersionAware.java | 76 + .../processors/cache/mvcc/MvccUtils.java | 882 ++++ .../processors/cache/mvcc/MvccVersion.java | 48 + .../cache/mvcc/MvccVersionAware.java | 76 + .../cache/mvcc/MvccVersionImpl.java | 193 + .../cache/mvcc/NoOpMvccProcessor.java | 215 + .../cache/mvcc/StaticMvccQueryTracker.java | 96 + .../processors/cache/mvcc/VacuumMetrics.java | 102 + .../cache/mvcc/VacuumMetricsReducer.java | 49 + .../processors/cache/mvcc/VacuumTask.java | 63 + .../mvcc/msg/MvccAckRequestQueryCntr.java | 130 + .../cache/mvcc/msg/MvccAckRequestQueryId.java | 130 + .../cache/mvcc/msg/MvccAckRequestTx.java | 210 + .../msg/MvccAckRequestTxAndQueryCntr.java | 123 + .../mvcc/msg/MvccAckRequestTxAndQueryId.java | 122 + .../mvcc/msg/MvccActiveQueriesMessage.java | 132 + .../cache/mvcc/msg/MvccFutureResponse.java | 130 + .../cache/mvcc/msg/MvccMessage.java | 35 + .../mvcc/msg/MvccQuerySnapshotRequest.java | 130 + .../cache/mvcc/msg/MvccSnapshotResponse.java | 320 ++ .../cache/mvcc/msg/MvccTxSnapshotRequest.java | 128 + .../cache/mvcc/msg/MvccWaitTxsRequest.java | 159 + .../processors/cache/mvcc/txlog/TxKey.java | 76 + .../processors/cache/mvcc/txlog/TxLog.java | 584 +++ .../processors/cache/mvcc/txlog/TxLogIO.java | 73 + .../cache/mvcc/txlog/TxLogInnerIO.java | 105 + .../cache/mvcc/txlog/TxLogLeafIO.java | 105 + .../cache/mvcc/txlog/TxLogTree.java | 62 + .../processors/cache/mvcc/txlog/TxRow.java | 44 + .../processors/cache/mvcc/txlog/TxState.java | 37 + .../cache/persistence/CacheDataRow.java | 3 +- .../persistence/CacheDataRowAdapter.java | 114 +- .../cache/persistence/CacheSearchRow.java | 3 +- .../DatabaseLifecycleListener.java | 54 + .../GridCacheDatabaseSharedManager.java | 39 +- .../persistence/GridCacheOffheapManager.java | 304 +- .../IgniteCacheDatabaseSharedManager.java | 32 +- .../cache/persistence/IndexStorageImpl.java | 2 +- .../cache/persistence/RowStore.java | 28 +- .../cache/persistence/Storable.java | 14 + .../file/FilePageStoreManager.java | 27 +- .../freelist/AbstractFreeList.java | 20 +- .../cache/persistence/freelist/FreeList.java | 12 + .../metastorage/MetastorageDataRow.java | 11 + .../metastorage/MetastorageTree.java | 2 +- .../cache/persistence/tree/BPlusTree.java | 1219 +++++- .../tree/io/AbstractDataPageIO.java | 35 +- .../cache/persistence/tree/io/DataPageIO.java | 484 ++- .../cache/persistence/tree/io/IOVersions.java | 7 + .../cache/persistence/tree/io/PageIO.java | 133 +- .../persistence/tree/io/SimpleDataPageIO.java | 6 - .../reader/StandaloneGridKernalContext.java | 6 + .../persistence/wal/record/RecordTypes.java | 3 + .../serializer/RecordDataV1Serializer.java | 82 + .../GridCacheDistributedQueryManager.java | 14 +- .../cache/query/GridCacheQueryAdapter.java | 132 +- .../cache/query/GridCacheQueryManager.java | 18 +- .../cache/query/GridCacheQueryRequest.java | 59 +- .../cache/query/IgniteQueryErrorCode.java | 20 + .../cache/query/SqlFieldsQueryEx.java | 39 + .../CacheContinuousQueryManager.java | 10 +- .../cache/transactions/IgniteInternalTx.java | 16 + .../transactions/IgniteTransactionsImpl.java | 4 +- .../cache/transactions/IgniteTxAdapter.java | 149 +- .../cache/transactions/IgniteTxHandler.java | 26 +- .../IgniteTxImplicitSingleStateImpl.java | 16 +- .../transactions/IgniteTxLocalAdapter.java | 311 +- .../cache/transactions/IgniteTxManager.java | 13 + .../cache/transactions/IgniteTxRemoteEx.java | 12 + .../IgniteTxRemoteSingleStateImpl.java | 5 + .../IgniteTxRemoteStateAdapter.java | 21 +- .../transactions/IgniteTxRemoteStateImpl.java | 10 + .../cache/transactions/IgniteTxState.java | 16 +- .../cache/transactions/IgniteTxStateImpl.java | 26 +- .../cache/tree/AbstractDataInnerIO.java | 112 +- .../cache/tree/AbstractDataLeafIO.java | 125 +- .../cache/tree/CacheDataRowStore.java | 35 +- .../processors/cache/tree/CacheDataTree.java | 82 +- .../cache/tree/CacheIdAwareDataInnerIO.java | 8 +- .../cache/tree/CacheIdAwareDataLeafIO.java | 8 +- .../processors/cache/tree/DataInnerIO.java | 11 - .../processors/cache/tree/DataLeafIO.java | 11 - .../processors/cache/tree/DataRow.java | 28 +- .../cache/tree/PendingEntriesTree.java | 2 +- .../processors/cache/tree/PendingRow.java | 3 +- .../processors/cache/tree/RowLinkIO.java | 67 +- .../processors/cache/tree/SearchRow.java | 31 + .../data/MvccCacheIdAwareDataInnerIO.java | 68 + .../mvcc/data/MvccCacheIdAwareDataLeafIO.java | 89 + .../cache/tree/mvcc/data/MvccDataInnerIO.java | 71 + .../cache/tree/mvcc/data/MvccDataLeafIO.java | 89 + .../cache/tree/mvcc/data/MvccDataRow.java | 263 ++ .../tree/mvcc/data/MvccUpdateDataRow.java | 485 +++ .../mvcc/data/MvccUpdateDataRowNative.java | 240 ++ .../tree/mvcc/data/MvccUpdateResult.java | 43 + .../cache/tree/mvcc/data/ResultType.java | 34 + .../mvcc/search/MvccFirstRowTreeClosure.java | 76 + .../mvcc/search/MvccLinkAwareSearchRow.java | 48 + .../tree/mvcc/search/MvccMaxSearchRow.java | 55 + .../tree/mvcc/search/MvccMinSearchRow.java | 55 + .../cache/tree/mvcc/search/MvccSearchRow.java | 70 + .../mvcc/search/MvccSnapshotSearchRow.java | 113 + .../tree/mvcc/search/MvccTreeClosure.java | 29 + .../processors/odbc/SqlStateCode.java | 3 + .../odbc/jdbc/JdbcBatchExecuteRequest.java | 27 +- .../odbc/jdbc/JdbcConnectionContext.java | 47 +- .../jdbc/JdbcOrderedBatchExecuteRequest.java | 5 +- .../odbc/jdbc/JdbcQueryExecuteRequest.java | 27 +- .../odbc/jdbc/JdbcRequestHandler.java | 65 +- .../odbc/jdbc/JdbcRequestHandlerWorker.java | 131 + .../odbc/odbc/OdbcConnectionContext.java | 21 +- .../odbc/odbc/OdbcMessageParser.java | 16 +- .../odbc/OdbcQueryExecuteBatchRequest.java | 15 +- .../odbc/odbc/OdbcQueryExecuteRequest.java | 14 +- .../odbc/odbc/OdbcRequestHandler.java | 73 +- .../odbc/odbc/OdbcRequestHandlerWorker.java | 131 + .../processors/query/EnlistOperation.java | 91 + .../processors/query/GridQueryIndexing.java | 41 +- .../processors/query/GridQueryProcessor.java | 52 +- .../processors/query/NestedTxMode.java | 60 + .../query/UpdateSourceIterator.java | 89 + .../messages/GridQueryNextPageResponse.java | 38 +- .../GridInternalSubscriptionProcessor.java | 18 + .../ignite/internal/sql/SqlKeyword.java | 20 + .../apache/ignite/internal/sql/SqlParser.java | 85 +- .../ignite/internal/sql/SqlParserUtils.java | 14 + .../command/SqlBeginTransactionCommand.java | 46 + .../command/SqlCommitTransactionCommand.java | 46 + .../SqlRollbackTransactionCommand.java | 46 + .../IgniteTxMvccVersionCheckedException.java | 47 + .../util/future/GridCompoundFuture.java | 4 +- .../TestRecordingCommunicationSpi.java | 26 +- .../cache/GridCacheTestEntryEx.java | 73 +- ...niteClientCacheInitializationFailTest.java | 20 +- ...cAbstractBasicCoordinatorFailoverTest.java | 685 +++ ...heMvccAbstractCoordinatorFailoverTest.java | 148 + .../mvcc/CacheMvccAbstractFeatureTest.java | 294 ++ .../cache/mvcc/CacheMvccAbstractTest.java | 2291 ++++++++++ .../mvcc/CacheMvccClusterRestartTest.java | 178 + .../CacheMvccConfigurationValidationTest.java | 191 + ...IteratorWithConcurrentTransactionTest.java | 64 + ...lEntriesWithConcurrentTransactionTest.java | 65 + .../mvcc/CacheMvccOperationChecksTest.java | 201 + ...vccPartitionedCoordinatorFailoverTest.java | 144 + .../cache/mvcc/CacheMvccProcessorTest.java | 97 + ...MvccReplicatedCoordinatorFailoverTest.java | 30 + ...canQueryWithConcurrentTransactionTest.java | 76 + ...MvccSizeWithConcurrentTransactionTest.java | 52 + .../cache/mvcc/CacheMvccTransactionsTest.java | 3674 +++++++++++++++++ .../cache/mvcc/CacheMvccVacuumTest.java | 158 + .../MemoryPolicyInitializationTest.java | 12 +- .../pagemem/NoOpPageStoreManager.java | 7 + .../database/BPlusTreeSelfTest.java | 377 +- .../database/CacheFreeListImplSelfTest.java | 55 + .../DataStreamProcessorMvccSeflTest.java | 68 + .../DataStreamProcessorSelfTest.java | 24 +- ...qlParserTransactionalKeywordsSelfTest.java | 96 + .../junits/GridAbstractTest.java | 8 + .../testsuites/IgniteCacheMvccTestSuite.java | 61 + .../development/utils/IgniteWalConverter.java | 5 +- .../query/h2/opt/GridH2SpatialIndex.java | 11 +- .../cache/query/GridCacheTwoStepQuery.java | 36 + .../query/h2/DhtResultSetEnlistFuture.java | 63 + .../query/h2/DmlStatementsProcessor.java | 352 +- .../query/h2/H2ConnectionWrapper.java | 52 +- .../processors/query/h2/H2FieldsIterator.java | 29 +- .../query/h2/H2KeyValueIterator.java | 2 +- .../query/h2/H2ResultSetIterator.java | 7 +- .../processors/query/h2/H2StatementCache.java | 78 +- .../processors/query/h2/IgniteH2Indexing.java | 983 +++-- .../query/h2/NearResultSetEnlistFuture.java | 62 + .../query/h2/PreparedStatementEx.java | 45 + .../query/h2/PreparedStatementExImpl.java | 648 +++ .../query/h2/RebuildIndexFromHashClosure.java | 47 + .../query/h2/ResultSetEnlistFuture.java | 136 + .../query/h2/ThreadLocalObjectPool.java | 103 + .../query/h2/database/H2PkHashIndex.java | 108 +- .../query/h2/database/H2RowFactory.java | 25 +- .../processors/query/h2/database/H2Tree.java | 112 +- .../h2/database/H2TreeFilterClosure.java | 109 + .../query/h2/database/H2TreeIndex.java | 160 +- .../database/io/AbstractH2ExtrasInnerIO.java | 164 + .../database/io/AbstractH2ExtrasLeafIO.java | 161 + .../h2/database/io/AbstractH2InnerIO.java | 74 + .../h2/database/io/AbstractH2LeafIO.java | 76 + .../query/h2/database/io/H2ExtrasInnerIO.java | 111 +- .../query/h2/database/io/H2ExtrasLeafIO.java | 107 +- .../query/h2/database/io/H2IOUtils.java | 94 + .../query/h2/database/io/H2InnerIO.java | 41 +- .../query/h2/database/io/H2LeafIO.java | 41 +- .../h2/database/io/H2MvccExtrasInnerIO.java | 55 + .../h2/database/io/H2MvccExtrasLeafIO.java | 54 + .../query/h2/database/io/H2MvccInnerIO.java | 58 + .../query/h2/database/io/H2MvccLeafIO.java | 58 + .../query/h2/database/io/H2RowLinkIO.java | 34 + .../query/h2/ddl/DdlStatementsProcessor.java | 27 + .../processors/query/h2/dml/DmlAstUtils.java | 12 +- .../processors/query/h2/dml/FastUpdate.java | 17 + .../processors/query/h2/dml/UpdatePlan.java | 233 ++ .../query/h2/dml/UpdatePlanBuilder.java | 118 +- .../query/h2/opt/GridH2IndexBase.java | 36 +- .../query/h2/opt/GridH2KeyRowOnheap.java | 11 + .../query/h2/opt/GridH2KeyValueRowOnheap.java | 21 +- .../query/h2/opt/GridH2MetaTable.java | 5 + .../query/h2/opt/GridH2PlainRowFactory.java | 17 +- .../query/h2/opt/GridH2QueryContext.java | 27 +- .../processors/query/h2/opt/GridH2Row.java | 50 +- .../query/h2/opt/GridH2RowDescriptor.java | 9 +- .../query/h2/opt/GridH2SearchRow.java | 32 + .../query/h2/opt/GridH2SearchRowAdapter.java | 27 +- .../processors/query/h2/opt/GridH2Table.java | 8 +- .../query/h2/sql/GridSqlQueryParser.java | 164 +- .../query/h2/sql/GridSqlQuerySplitter.java | 18 +- .../query/h2/sql/GridSqlSelect.java | 20 + .../h2/twostep/GridMapQueryExecutor.java | 334 +- .../h2/twostep/GridMergeIndexIterator.java | 17 +- .../h2/twostep/GridReduceQueryExecutor.java | 325 +- .../query/h2/twostep/GridResultPage.java | 22 +- .../query/h2/twostep/MapQueryResults.java | 18 +- .../query/h2/twostep/ReduceQueryRun.java | 111 +- .../h2/twostep/msg/GridH2QueryRequest.java | 115 +- .../msg/GridH2SelectForUpdateTxDetails.java | 299 ++ .../msg/GridH2ValueMessageFactory.java | 12 +- .../cache/DdlTransactionSelfTest.java | 335 ++ .../IgniteCacheQueryH2IndexingLeakTest.java | 10 +- .../cache/index/AbstractSchemaSelfTest.java | 102 +- ...amicColumnsAbstractConcurrentSelfTest.java | 13 +- .../index/DynamicIndexAbstractSelfTest.java | 7 + .../index/H2ConnectionLeaksSelfTest.java | 8 +- .../index/H2DynamicIndexAbstractSelfTest.java | 49 - .../cache/index/H2DynamicTableSelfTest.java | 16 +- ...ctionsComandsWithMvccDisabledSelfTest.java | 83 + .../cache/index/SqlTransactionsSelfTest.java | 421 ++ ...vccAbstractSqlCoordinatorFailoverTest.java | 138 + .../mvcc/CacheMvccBackupsAbstractTest.java | 808 ++++ .../cache/mvcc/CacheMvccBulkLoadTest.java | 123 + .../cache/mvcc/CacheMvccDmlSimpleTest.java | 200 + ...atorWithConcurrentJdbcTransactionTest.java | 28 + ...riesWithConcurrentJdbcTransactionTest.java | 29 + .../mvcc/CacheMvccPartitionedBackupsTest.java | 30 + ...ccPartitionedSelectForUpdateQueryTest.java | 44 + ...PartitionedSqlCoordinatorFailoverTest.java | 143 + .../CacheMvccPartitionedSqlQueriesTest.java | 30 + .../CacheMvccPartitionedSqlTxQueriesTest.java | 30 + ...artitionedSqlTxQueriesWithReducerTest.java | 30 + .../mvcc/CacheMvccReplicatedBackupsTest.java | 30 + ...vccReplicatedSelectForUpdateQueryTest.java | 30 + ...cReplicatedSqlCoordinatorFailoverTest.java | 30 + .../CacheMvccReplicatedSqlQueriesTest.java | 30 + .../CacheMvccReplicatedSqlTxQueriesTest.java | 242 ++ ...ReplicatedSqlTxQueriesWithReducerTest.java | 30 + ...ueryWithConcurrentJdbcTransactionTest.java | 29 + ...eMvccSelectForUpdateQueryAbstractTest.java | 374 ++ ...SizeWithConcurrentJdbcTransactionTest.java | 28 + .../mvcc/CacheMvccSqlQueriesAbstractTest.java | 1611 ++++++++ .../CacheMvccSqlTxQueriesAbstractTest.java | 1832 ++++++++ ...ccSqlTxQueriesWithReducerAbstractTest.java | 829 ++++ .../mvcc/CacheMvccStreamingInsertTest.java | 122 + .../query/h2/GridIndexRebuildSelfTest.java | 350 ++ .../h2/GridIndexingSpiAbstractSelfTest.java | 4 +- .../query/h2/H2StatementCacheSelfTest.java | 83 + .../query/h2/PreparedStatementExSelfTest.java | 61 + .../h2/ThreadLocalObjectPoolSelfTest.java | 113 + .../query/h2/sql/GridQueryParsingTest.java | 1 + .../IgniteCacheMvccSqlTestSuite.java | 73 + .../IgniteCacheQuerySelfTestSuite.java | 20 + .../processors/cache/mvcc/mvcc_person.csv | 2 + .../cache/mvcc/mvcc_person_broken.csv | 2 + .../processors/cache/jta/CacheJtaManager.java | 1 + modules/platforms/cpp/odbc-test/Makefile.am | 4 +- .../config/queries-transaction-32.xml | 50 + .../odbc-test/config/queries-transaction.xml | 32 + .../odbc-test/project/vs/odbc-test.vcxproj | 4 + .../project/vs/odbc-test.vcxproj.filters | 12 + .../cpp/odbc-test/src/sql_get_info_test.cpp | 8 +- .../cpp/odbc-test/src/transaction_test.cpp | 738 ++++ modules/platforms/cpp/odbc/Makefile.am | 1 + .../ignite/odbc/config/configuration.h | 32 + .../odbc/config/connection_string_parser.h | 3 + .../cpp/odbc/include/ignite/odbc/connection.h | 21 +- .../ignite/odbc/diagnostic/diagnosable.h | 7 + .../odbc/diagnostic/diagnosable_adapter.h | 11 +- .../odbc/include/ignite/odbc/environment.h | 15 + .../cpp/odbc/include/ignite/odbc/message.h | 13 +- .../odbc/include/ignite/odbc/nested_tx_mode.h | 81 + .../include/ignite/odbc/protocol_version.h | 2 +- .../odbc/system/ui/dsn_configuration_window.h | 8 + .../system/ui/dsn_configuration_window.cpp | 56 +- .../cpp/odbc/project/vs/odbc.vcxproj | 2 + .../cpp/odbc/project/vs/odbc.vcxproj.filters | 6 + .../cpp/odbc/src/config/configuration.cpp | 29 +- .../cpp/odbc/src/config/connection_info.cpp | 8 +- .../src/config/connection_string_parser.cpp | 19 + modules/platforms/cpp/odbc/src/connection.cpp | 96 +- .../src/diagnostic/diagnosable_adapter.cpp | 5 + modules/platforms/cpp/odbc/src/dsn_config.cpp | 5 + .../platforms/cpp/odbc/src/environment.cpp | 53 +- modules/platforms/cpp/odbc/src/message.cpp | 18 +- .../platforms/cpp/odbc/src/nested_tx_mode.cpp | 80 + modules/platforms/cpp/odbc/src/odbc.cpp | 2 + .../cpp/odbc/src/query/batch_query.cpp | 4 +- .../cpp/odbc/src/query/data_query.cpp | 2 +- .../config/mvcc/benchmark-mvcc-messages.sh | 98 + .../config/mvcc/benchmark-mvcc-processor.sh | 95 + .../mvcc/benchmark-mvcc-updates-contention.sh | 96 + .../mvcc/benchmark-thin-native.properties | 124 + ...nchmark-jdbc-thin-inmemory-mvcc.properties | 105 + .../yardstick/IgniteBenchmarkArguments.java | 38 + .../apache/ignite/yardstick/IgniteNode.java | 2 + .../yardstick/jdbc/AbstractJdbcBenchmark.java | 2 +- .../jdbc/AbstractNativeBenchmark.java | 2 +- .../ignite/yardstick/jdbc/JdbcUtils.java | 11 +- .../jdbc/NativeJavaApiPutRemoveBenchmark.java | 52 + .../AbstractDistributedMvccBenchmark.java | 101 + .../jdbc/mvcc/MvccProcessorBenchmark.java | 61 + .../mvcc/MvccUpdateContentionBenchmark.java | 77 + .../upload/AbstractNativeBenchmark.java | 14 +- .../upload/AbstractUploadBenchmark.java | 9 +- .../yardstick/upload/model/QueryFactory.java | 30 +- 475 files changed, 56365 insertions(+), 2860 deletions(-) create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsAbstractComplexSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientAutoCommitComplexSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerAutoCommitComplexSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccMarkUpdatedRecord.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateNewTxStateHintRecord.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateTxStateHintRecord.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CompoundLockFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/DhtLockFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ExceptionAware.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsUpdateCountersMap.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryResultsEnlistFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/AckCoordinatorOnRollback.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishAndAckFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/TxTopologyVersionFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCoordinator.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccDiscoveryData.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccEmptyLongList.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccLongList.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccPreviousCoordinatorQueries.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTracker.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshot.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotResponseListener.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotWithoutTxs.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersion.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionAware.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionImpl.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/NoOpMvccProcessor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/StaticMvccQueryTracker.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetrics.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetricsReducer.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryCntr.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryId.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTx.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryCntr.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccActiveQueriesMessage.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccFutureResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccMessage.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccQuerySnapshotRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccWaitTxsRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxKey.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLog.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogIO.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogInnerIO.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogLeafIO.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogTree.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxState.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataInnerIO.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataLeafIO.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataInnerIO.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataLeafIO.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRowNative.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/ResultType.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccFirstRowTreeClosure.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccLinkAwareSearchRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMaxSearchRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMinSearchRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSearchRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSnapshotSearchRow.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccTreeClosure.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandlerWorker.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandlerWorker.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/EnlistOperation.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/NestedTxMode.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/UpdateSourceIterator.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBeginTransactionCommand.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCommitTransactionCommand.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlRollbackTransactionCommand.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxMvccVersionCheckedException.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractCoordinatorFailoverTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentTransactionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentTransactionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccOperationChecksTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedCoordinatorFailoverTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccProcessorTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedCoordinatorFailoverTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentTransactionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentTransactionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSeflTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserTransactionalKeywordsSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DhtResultSetEnlistFuture.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/NearResultSetEnlistFuture.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementEx.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExImpl.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/RebuildIndexFromHashClosure.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ResultSetEnlistFuture.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasInnerIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasLeafIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2InnerIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2LeafIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2IOUtils.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasInnerIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasLeafIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccInnerIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccLeafIO.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRow.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2SelectForUpdateTxDetails.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlCoordinatorFailoverTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBulkLoadTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccDmlSimpleTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentJdbcTransactionTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedBackupsTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSelectForUpdateQueryTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlQueriesTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesWithReducerTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedBackupsTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSelectForUpdateQueryTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlCoordinatorFailoverTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlQueriesTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesWithReducerTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentJdbcTransactionTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentJdbcTransactionTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesAbstractTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccStreamingInsertTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2StatementCacheSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java create mode 100644 modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person.csv create mode 100644 modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person_broken.csv create mode 100644 modules/platforms/cpp/odbc-test/config/queries-transaction-32.xml create mode 100644 modules/platforms/cpp/odbc-test/config/queries-transaction.xml create mode 100644 modules/platforms/cpp/odbc-test/src/transaction_test.cpp create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/nested_tx_mode.h create mode 100644 modules/platforms/cpp/odbc/src/nested_tx_mode.cpp create mode 100644 modules/yardstick/config/mvcc/benchmark-mvcc-messages.sh create mode 100644 modules/yardstick/config/mvcc/benchmark-mvcc-processor.sh create mode 100644 modules/yardstick/config/mvcc/benchmark-mvcc-updates-contention.sh create mode 100644 modules/yardstick/config/mvcc/benchmark-thin-native.properties create mode 100644 modules/yardstick/config/upload/benchmark-jdbc-thin-inmemory-mvcc.properties create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeJavaApiPutRemoveBenchmark.java create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/AbstractDistributedMvccBenchmark.java create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccProcessorBenchmark.java create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccUpdateContentionBenchmark.java diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java index 7ed84cbd8d93e..e80e13d52d4b3 100644 --- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java +++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/tree/BPlusTreeBenchmark.java @@ -192,7 +192,7 @@ protected static class TestTree extends BPlusTree { } /** {@inheritDoc} */ - @Override protected Long getRow(BPlusIO io, long pageAddr, int idx, Object ignore) + @Override public Long getRow(BPlusIO io, long pageAddr, int idx, Object ignore) throws IgniteCheckedException { assert io.canGetRow() : io; diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java index 9e11efc7280c5..bc545ac70f1fa 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStreamingSelfTest.java @@ -34,6 +34,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.jdbc.thin.JdbcThinAbstractSelfTest; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -47,7 +48,7 @@ /** * Data streaming test. */ -public class JdbcStreamingSelfTest extends GridCommonAbstractTest { +public class JdbcStreamingSelfTest extends JdbcThinAbstractSelfTest { /** JDBC URL. */ private static final String BASE_URL = CFG_URL_PREFIX + "cache=default@modules/clients/src/test/config/jdbc-config.xml"; 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 889a551bc0bc8..275040f81619f 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 @@ -48,6 +48,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinComplexQuerySelfTest; import org.apache.ignite.jdbc.thin.JdbcThinConnectionMultipleAddressesTest; +import org.apache.ignite.jdbc.thin.JdbcThinConnectionMvccEnabledSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinConnectionSSLTest; import org.apache.ignite.jdbc.thin.JdbcThinConnectionSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinDataSourceSelfTest; @@ -76,6 +77,11 @@ import org.apache.ignite.jdbc.thin.JdbcThinStreamingNotOrderedSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinStreamingOrderedSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinTcpIoTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientAutoCommitComplexSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientNoAutoCommitComplexSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerAutoCommitComplexSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerNoAutoCommitComplexSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSkipReducerOnUpdateSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinWalModeChangeSelfTest; @@ -146,6 +152,7 @@ public static TestSuite suite() throws Exception { // New thin JDBC suite.addTest(new TestSuite(JdbcThinConnectionSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinConnectionMvccEnabledSelfTest.class)); suite.addTest(new TestSuite(JdbcThinConnectionMultipleAddressesTest.class)); suite.addTest(new TestSuite(JdbcThinTcpIoTest.class)); suite.addTest(new TestSuite(JdbcThinConnectionSSLTest.class)); @@ -197,6 +204,13 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(JdbcThinComplexDmlDdlSkipReducerOnUpdateSelfTest.class)); suite.addTest(new TestSuite(JdbcThinComplexDmlDdlCustomSchemaSelfTest.class)); + // Transactions + suite.addTest(new TestSuite(JdbcThinTransactionsSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsClientAutoCommitComplexSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsServerAutoCommitComplexSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinLocalQueriesSelfTest.class)); // Various commands. diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractSelfTest.java index 2ba36c369c227..6d5f59a12c234 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractSelfTest.java @@ -112,7 +112,7 @@ interface RunnableX { * @param params Connection parameters. * @return Thin JDBC connection to specified node. */ - static Connection connect(IgniteEx node, String params) throws SQLException { + protected Connection connect(IgniteEx node, String params) throws SQLException { Collection recs = node.context().ports().records(); GridPortRecord cliLsnrRec = null; @@ -141,7 +141,7 @@ static Connection connect(IgniteEx node, String params) throws SQLException { * @return Result set. * @throws RuntimeException if failed. */ - static List> execute(Connection conn, String sql, Object... args) throws SQLException { + protected List> execute(Connection conn, String sql, Object... args) throws SQLException { try (PreparedStatement s = conn.prepareStatement(sql)) { for (int i = 0; i < args.length; i++) s.setObject(i + 1, args[i]); diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java new file mode 100644 index 0000000000000..051d1d29be833 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.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.jdbc.thin; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Savepoint; +import java.util.concurrent.Callable; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; +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.GridStringLogger; +import org.apache.ignite.testframework.GridTestUtils; +import org.jetbrains.annotations.NotNull; + +import static java.sql.Connection.TRANSACTION_NONE; +import static java.sql.Connection.TRANSACTION_READ_COMMITTED; +import static java.sql.Connection.TRANSACTION_READ_UNCOMMITTED; +import static java.sql.Connection.TRANSACTION_REPEATABLE_READ; +import static java.sql.Connection.TRANSACTION_SERIALIZABLE; + +/** + * Connection test. + */ +@SuppressWarnings("ThrowableNotThrown") +public class JdbcThinConnectionMvccEnabledSelfTest extends JdbcThinAbstractSelfTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String URL = "jdbc:ignite:thin://127.0.0.1"; + + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME)); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + + cfg.setMarshaller(new BinaryMarshaller()); + + cfg.setGridLogger(new GridStringLogger()); + + cfg.setMvccEnabled(true); + + return cfg; + } + + /** + * @param name Cache name. + * @return Cache configuration. + * @throws Exception In case of error. + */ + private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception { + CacheConfiguration cfg = defaultCacheConfiguration(); + + cfg.setName(name); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + + /** + * @throws Exception If failed. + */ + public void testMetadataDefaults() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + DatabaseMetaData meta = conn.getMetaData(); + + assertEquals(TRANSACTION_REPEATABLE_READ, meta.getDefaultTransactionIsolation()); + assertTrue(meta.supportsTransactions()); + + assertFalse(meta.supportsTransactionIsolationLevel(TRANSACTION_NONE)); + assertFalse(meta.supportsTransactionIsolationLevel(TRANSACTION_READ_UNCOMMITTED)); + assertFalse(meta.supportsTransactionIsolationLevel(TRANSACTION_READ_COMMITTED)); + assertTrue(meta.supportsTransactionIsolationLevel(TRANSACTION_REPEATABLE_READ)); + assertFalse(meta.supportsTransactionIsolationLevel(TRANSACTION_SERIALIZABLE)); + } + } + + /** + * @throws Exception If failed. + */ + public void testGetSetAutoCommit() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + assertTrue(conn.getMetaData().supportsTransactions()); + + assertTrue(conn.getAutoCommit()); + + conn.setAutoCommit(false); + + assertFalse(conn.getAutoCommit()); + + conn.setAutoCommit(true); + + assertTrue(conn.getAutoCommit()); + + conn.close(); + + // Exception when called on closed connection + checkConnectionClosed(new RunnableX() { + @Override public void run() throws Exception { + conn.setAutoCommit(true); + } + }); + } + } + + /** + * @throws Exception If failed. + */ + public void testCommit() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + assertTrue(conn.getMetaData().supportsTransactions()); + + // Should not be called in auto-commit mode + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.commit(); + + return null; + } + }, + SQLException.class, + "Transaction cannot be committed explicitly in auto-commit mode" + ); + + conn.setAutoCommit(false); + + conn.commit(); + + conn.close(); + + // Exception when called on closed connection + checkConnectionClosed(new RunnableX() { + @Override public void run() throws Exception { + conn.commit(); + } + }); + } + } + + /** + * @throws Exception If failed. + */ + public void testRollback() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + assertTrue(conn.getMetaData().supportsTransactions()); + + // Should not be called in auto-commit mode + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.rollback(); + + return null; + } + }, + SQLException.class, + "Transaction cannot be rolled back explicitly in auto-commit mode." + ); + + conn.setAutoCommit(false); + + conn.rollback(); + + conn.close(); + + // Exception when called on closed connection + checkConnectionClosed(new RunnableX() { + @Override public void run() throws Exception { + conn.rollback(); + } + }); + } + } + + /** + * @throws Exception If failed. + */ + public void testSetSavepoint() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + assert !conn.getMetaData().supportsSavepoints(); + + // Disallowed in auto-commit mode + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setSavepoint(); + + return null; + } + }, + SQLException.class, + "Savepoint cannot be set in auto-commit mode" + ); + + conn.setAutoCommit(false); + + // Unsupported + checkNotSupported(new RunnableX() { + @Override public void run() throws Exception { + conn.setSavepoint(); + } + }); + + conn.close(); + + checkConnectionClosed(new RunnableX() { + @Override public void run() throws Exception { + conn.setSavepoint(); + } + }); + } + } + + /** + * @throws Exception If failed. + */ + public void testSetSavepointName() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + assert !conn.getMetaData().supportsSavepoints(); + + // Invalid arg + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setSavepoint(null); + + return null; + } + }, + SQLException.class, + "Savepoint name cannot be null" + ); + + final String name = "savepoint"; + + // Disallowed in auto-commit mode + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setSavepoint(name); + + return null; + } + }, + SQLException.class, + "Savepoint cannot be set in auto-commit mode" + ); + + conn.setAutoCommit(false); + + // Unsupported + checkNotSupported(new RunnableX() { + @Override public void run() throws Exception { + conn.setSavepoint(name); + } + }); + + conn.close(); + + checkConnectionClosed(new RunnableX() { + @Override public void run() throws Exception { + conn.setSavepoint(name); + } + }); + } + } + + /** + * @throws Exception If failed. + */ + public void testRollbackSavePoint() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + assert !conn.getMetaData().supportsSavepoints(); + + // Invalid arg + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.rollback(null); + + return null; + } + }, + SQLException.class, + "Invalid savepoint" + ); + + final Savepoint savepoint = getFakeSavepoint(); + + // Disallowed in auto-commit mode + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.rollback(savepoint); + + return null; + } + }, + SQLException.class, + "Auto-commit mode" + ); + + conn.setAutoCommit(false); + + // Unsupported + checkNotSupported(new RunnableX() { + @Override public void run() throws Exception { + conn.rollback(savepoint); + } + }); + + conn.close(); + + checkConnectionClosed(new RunnableX() { + @Override public void run() throws Exception { + conn.rollback(savepoint); + } + }); + } + } + + /** + * @return Savepoint. + */ + private Savepoint getFakeSavepoint() { + return new Savepoint() { + @Override public int getSavepointId() throws SQLException { + return 100; + } + + @Override public String getSavepointName() { + return "savepoint"; + } + }; + } +} \ No newline at end of file diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java index 38bcab2824388..80397e65e7aea 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java @@ -17,6 +17,9 @@ package org.apache.ignite.jdbc.thin; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.DriverManager; @@ -38,12 +41,16 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.jdbc.thin.ConnectionProperties; +import org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl; import org.apache.ignite.internal.jdbc.thin.JdbcThinConnection; import org.apache.ignite.internal.jdbc.thin.JdbcThinTcpIo; +import org.apache.ignite.internal.util.HostAndPortRange; 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.GridStringLogger; import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.NotNull; @@ -58,6 +65,7 @@ import static java.sql.ResultSet.TYPE_FORWARD_ONLY; import static java.sql.Statement.NO_GENERATED_KEYS; import static java.sql.Statement.RETURN_GENERATED_KEYS; +import static org.apache.ignite.configuration.ClientConnectorConfiguration.DFLT_PORT; /** * Connection test. @@ -93,6 +101,8 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest { cfg.setMarshaller(new BinaryMarshaller()); + cfg.setGridLogger(new GridStringLogger()); + return cfg; } @@ -970,11 +980,18 @@ public void testGetSetAutoCommit() throws Exception { try (Connection conn = DriverManager.getConnection(URL)) { assertTrue(conn.getAutoCommit()); - conn.setAutoCommit(false); - - assertFalse(conn.getAutoCommit()); + // Cannot disable autocommit when MVCC is disabled. + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setAutoCommit(false); - conn.setAutoCommit(true); + return null; + } + }, + SQLException.class, + "MVCC must be enabled in order to invoke transactional operation: COMMIT" + ); assertTrue(conn.getAutoCommit()); @@ -994,8 +1011,6 @@ public void testGetSetAutoCommit() throws Exception { */ public void testCommit() throws Exception { try (Connection conn = DriverManager.getConnection(URL)) { - assert !conn.getMetaData().supportsTransactions(); - // Should not be called in auto-commit mode GridTestUtils.assertThrows(log, new Callable() { @@ -1009,9 +1024,33 @@ public void testCommit() throws Exception { "Transaction cannot be committed explicitly in auto-commit mode" ); - conn.setAutoCommit(false); + // Cannot disable autocommit when MVCC is disabled. + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setAutoCommit(false); + + return null; + } + }, + SQLException.class, + "MVCC must be enabled in order to invoke transactional operation: COMMIT" + ); + + assertTrue(conn.getAutoCommit()); - conn.commit(); + // Should not be called in auto-commit mode + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.commit(); + + return null; + } + }, + SQLException.class, + "Transaction cannot be committed explicitly in auto-commit mode." + ); conn.close(); @@ -1029,8 +1068,6 @@ public void testCommit() throws Exception { */ public void testRollback() throws Exception { try (Connection conn = DriverManager.getConnection(URL)) { - assert !conn.getMetaData().supportsTransactions(); - // Should not be called in auto-commit mode GridTestUtils.assertThrows(log, new Callable() { @@ -1041,12 +1078,23 @@ public void testRollback() throws Exception { } }, SQLException.class, - "Transaction cannot rollback in auto-commit mode" + "Transaction cannot be rolled back explicitly in auto-commit mode." ); - conn.setAutoCommit(false); + // Cannot disable autocommit when MVCC is disabled. + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setAutoCommit(false); - conn.rollback(); + return null; + } + }, + SQLException.class, + "MVCC must be enabled in order to invoke transactional operation: COMMIT" + ); + + assertTrue(conn.getAutoCommit()); conn.close(); @@ -1138,8 +1186,6 @@ public void testGetSetCatalog() throws Exception { */ public void testGetSetTransactionIsolation() throws Exception { try (Connection conn = DriverManager.getConnection(URL)) { - assert !conn.getMetaData().supportsTransactions(); - // Invalid parameter value GridTestUtils.assertThrows(log, new Callable() { @@ -1346,14 +1392,20 @@ public void testSetSavepoint() throws Exception { "Savepoint cannot be set in auto-commit mode" ); - conn.setAutoCommit(false); + // Cannot disable autocommit when MVCC is disabled. + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setAutoCommit(false); - // Unsupported - checkNotSupported(new RunnableX() { - @Override public void run() throws Exception { - conn.setSavepoint(); - } - }); + return null; + } + }, + SQLException.class, + "MVCC must be enabled in order to invoke transactional operation: COMMIT" + ); + + assertTrue(conn.getAutoCommit()); conn.close(); @@ -1400,14 +1452,20 @@ public void testSetSavepointName() throws Exception { "Savepoint cannot be set in auto-commit mode" ); - conn.setAutoCommit(false); + // Cannot disable autocommit when MVCC is disabled. + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setAutoCommit(false); - // Unsupported - checkNotSupported(new RunnableX() { - @Override public void run() throws Exception { - conn.setSavepoint(name); - } - }); + return null; + } + }, + SQLException.class, + "MVCC must be enabled in order to invoke transactional operation: COMMIT" + ); + + assertTrue(conn.getAutoCommit()); conn.close(); @@ -1454,14 +1512,20 @@ public void testRollbackSavePoint() throws Exception { "Auto-commit mode" ); - conn.setAutoCommit(false); + // Cannot disable autocommit when MVCC is disabled. + GridTestUtils.assertThrows(log, + new Callable() { + @Override public Object call() throws Exception { + conn.setAutoCommit(false); - // Unsupported - checkNotSupported(new RunnableX() { - @Override public void run() throws Exception { - conn.rollback(savepoint); - } - }); + return null; + } + }, + SQLException.class, + "MVCC must be enabled in order to invoke transactional operation: COMMIT" + ); + + assertTrue(conn.getAutoCommit()); conn.close(); @@ -1897,6 +1961,58 @@ public void testGetSetNetworkTimeout() throws Exception { } } + /** + * Test that attempting to supply invalid nested TX mode to driver fails on the client. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testInvalidNestedTxMode() { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + DriverManager.getConnection(URL + "/?nestedTransactionsMode=invalid"); + + return null; + } + }, SQLException.class, "Invalid nested transactions handling mode"); + } + + /** + * Test that attempting to send unexpected name of nested TX mode to server on handshake yields an error. + * We have to do this without explicit {@link Connection} as long as there's no other way to bypass validation and + * supply a malformed {@link ConnectionProperties} to {@link JdbcThinTcpIo}. + */ + @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "ThrowFromFinallyBlock"}) + public void testInvalidNestedTxModeOnServerSide() throws SQLException, NoSuchMethodException, + IllegalAccessException, InvocationTargetException, InstantiationException, IOException { + ConnectionPropertiesImpl connProps = new ConnectionPropertiesImpl(); + + connProps.setAddresses(new HostAndPortRange[]{new HostAndPortRange("127.0.0.1", DFLT_PORT, DFLT_PORT)}); + + connProps.nestedTxMode("invalid"); + + Constructor ctor = JdbcThinTcpIo.class.getDeclaredConstructor(ConnectionProperties.class); + + boolean acc = ctor.isAccessible(); + + ctor.setAccessible(true); + + final JdbcThinTcpIo io = (JdbcThinTcpIo)ctor.newInstance(connProps); + + try { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + io.start(); + + return null; + } + }, SQLException.class, "err=Invalid nested transactions handling mode: invalid"); + } + finally { + io.close(); + + ctor.setAccessible(acc); + } + } + /** */ public void testSslClientAndPlainServer() { diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingAbstractSelfTest.java index 70046356b210b..c83977c692d19 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingAbstractSelfTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.processors.query.SqlClientContext; @@ -77,9 +78,19 @@ public abstract class JdbcThinStreamingAbstractSelfTest extends JdbcStreamingSel super.afterTest(); } - /** {@inheritDoc} */ + /** {@inheritDoc} */ + @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { + Connection c = connect(grid(0), null); + + execute(c, "SET STREAMING 1 BATCH_SIZE " + batchSize + " ALLOW_OVERWRITE " + (allowOverwrite ? 1 : 0) + + " PER_NODE_BUFFER_SIZE 1000 FLUSH_FREQUENCY " + flushFreq); + + return c; + } + + /** {@inheritDoc} */ @Override protected Connection createOrdinaryConnection() throws SQLException { - return JdbcThinAbstractSelfTest.connect(grid(0), null); + return connect(grid(0), null); } /** @@ -495,11 +506,11 @@ static final class IndexingWithContext extends IgniteH2Indexing { /** {@inheritDoc} */ @Override public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - @Nullable SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, + @Nullable SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, MvccQueryTracker tracker, GridQueryCancel cancel) { IndexingWithContext.cliCtx = cliCtx; - return super.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, cancel); + return super.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, tracker, cancel); } } } \ No newline at end of file diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingNotOrderedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingNotOrderedSelfTest.java index b91258f8a4eb9..b0788e74f1bc8 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingNotOrderedSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingNotOrderedSelfTest.java @@ -25,7 +25,7 @@ public class JdbcThinStreamingNotOrderedSelfTest extends JdbcThinStreamingAbstractSelfTest { /** {@inheritDoc} */ @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { - Connection c = JdbcThinAbstractSelfTest.connect(grid(0), null); + Connection c = connect(grid(0), null); execute(c, "SET STREAMING 1 BATCH_SIZE " + batchSize + " ALLOW_OVERWRITE " + (allowOverwrite ? 1 : 0) diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingOrderedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingOrderedSelfTest.java index b615f8cd9614e..c116a7dc6c48a 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingOrderedSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingOrderedSelfTest.java @@ -25,7 +25,7 @@ public class JdbcThinStreamingOrderedSelfTest extends JdbcThinStreamingAbstractSelfTest { /** {@inheritDoc} */ @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { - Connection c = JdbcThinAbstractSelfTest.connect(grid(0), null); + Connection c = connect(grid(0), null); execute(c, "SET STREAMING 1 BATCH_SIZE " + batchSize + " ALLOW_OVERWRITE " + (allowOverwrite ? 1 : 0) diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsAbstractComplexSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsAbstractComplexSelfTest.java new file mode 100644 index 0000000000000..28c65a97cd6b6 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsAbstractComplexSelfTest.java @@ -0,0 +1,1058 @@ +/* + * 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.jdbc.thin; + +import java.sql.BatchUpdateException; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +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.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Test to check various transactional scenarios. + */ +public abstract class JdbcThinTransactionsAbstractComplexSelfTest extends JdbcThinAbstractSelfTest { + /** Client node index. */ + final static int CLI_IDX = 1; + + /** + * Closure to perform ordinary delete after repeatable read. + */ + private final IgniteInClosure afterReadDel = new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where firstname = 'John'"); + } + }; + + /** + * Closure to perform fast delete after repeatable read. + */ + private final IgniteInClosure afterReadFastDel = new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where id = 1"); + } + }; + + /** + * Closure to perform ordinary update after repeatable read. + */ + private final IgniteInClosure afterReadUpdate = new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "UPDATE \"Person\".Person set firstname = 'Joe' where firstname = 'John'"); + } + }; + + /** + * Closure to perform ordinary delete and rollback after repeatable read. + */ + private final IgniteInClosure afterReadDelAndRollback = new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where firstname = 'John'"); + + rollback(conn); + } + }; + + /** + * Closure to perform fast delete after repeatable read. + */ + private final IgniteInClosure afterReadFastDelAndRollback = new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where id = 1"); + + rollback(conn); + } + }; + + /** + * Closure to perform ordinary update and rollback after repeatable read. + */ + private final IgniteInClosure afterReadUpdateAndRollback = new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "UPDATE \"Person\".Person set firstname = 'Joe' where firstname = 'John'"); + + rollback(conn); + } + }; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String testIgniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(testIgniteInstanceName); + + cfg.setMvccEnabled(true); + + CacheConfiguration ccfg = new CacheConfiguration<>("Person"); + + ccfg.setIndexedTypes(Integer.class, Person.class); + + ccfg.getQueryEntities().iterator().next().setKeyFieldName("id"); + + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + ccfg.setCacheMode(CacheMode.PARTITIONED); + + cfg.setCacheConfiguration(ccfg); + + // Let the node with index 1 be client node. + cfg.setClientMode(F.eq(testIgniteInstanceName, getTestIgniteInstanceName(CLI_IDX))); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + execute("ALTER TABLE \"Person\".person add if not exists cityid int"); + + execute("ALTER TABLE \"Person\".person add if not exists companyid int"); + + execute("CREATE TABLE City (id int primary key, name varchar, population int) WITH " + + "\"atomicity=transactional,template=partitioned,backups=3,cache_name=City\""); + + execute("CREATE TABLE Company (id int, \"cityid\" int, name varchar, primary key (id, \"cityid\")) WITH " + + "\"atomicity=transactional,template=partitioned,backups=1,wrap_value=false,affinity_key=cityid," + + "cache_name=Company\""); + + execute("CREATE TABLE Product (id int primary key, name varchar, companyid int) WITH " + + "\"atomicity=transactional,template=partitioned,backups=2,cache_name=Product\""); + + execute("CREATE INDEX IF NOT EXISTS prodidx ON Product(companyid)"); + + execute("CREATE INDEX IF NOT EXISTS persidx ON \"Person\".person(cityid)"); + + insertPerson(1, "John", "Smith", 1, 1); + + insertPerson(2, "Mike", "Johns", 1, 2); + + insertPerson(3, "Sam", "Jules", 2, 2); + + insertPerson(4, "Alex", "Pope", 2, 3); + + insertPerson(5, "Peter", "Williams", 2, 3); + + insertCity(1, "Los Angeles", 5000); + + insertCity(2, "Seattle", 1500); + + insertCity(3, "New York", 12000); + + insertCity(4, "Cupertino", 400); + + insertCompany(1, "Microsoft", 2); + + insertCompany(2, "Google", 3); + + insertCompany(3, "Facebook", 1); + + insertCompany(4, "Uber", 1); + + insertCompany(5, "Apple", 4); + + insertProduct(1, "Search", 2); + + insertProduct(2, "Windows", 1); + + insertProduct(3, "Mac", 5); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + + startGrid(1); + + startGrid(2); + + startGrid(3); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + execute("DELETE FROM \"Person\".Person"); + + execute("DROP TABLE City"); + + execute("DROP TABLE Company"); + + execute("DROP TABLE Product"); + + super.afterTest(); + } + + /** + * + */ + public void testSingleDmlStatement() throws SQLException { + insertPerson(6, "John", "Doe", 2, 2); + + assertEquals(Collections.singletonList(l(6, "John", "Doe", 2, 2)), + execute("SELECT * FROM \"Person\".Person where id = 6")); + } + + /** + * + */ + public void testMultipleDmlStatements() throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + insertPerson(conn, 6, "John", "Doe", 2, 2); + + // https://issues.apache.org/jira/browse/IGNITE-6938 - we can only see results of + // UPDATE of what we have not inserted ourselves. + execute(conn, "UPDATE \"Person\".person SET lastname = 'Jameson' where lastname = 'Jules'"); + + execute(conn, "DELETE FROM \"Person\".person where id = 5"); + } + }); + + assertEquals(l( + l(3, "Sam", "Jameson", 2, 2), + l(6, "John", "Doe", 2, 2) + ), execute("SELECT * FROM \"Person\".Person where id = 3 or id >= 5 order by id")); + } + + /** + * + */ + public void testBatchDmlStatements() throws SQLException { + doBatchedInsert(); + + assertEquals(l( + l(6, "John", "Doe", 2, 2), + l(7, "Mary", "Lee", 1, 3) + ), execute("SELECT * FROM \"Person\".Person where id > 5 order by id")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testBatchDmlStatementsIntermediateFailure() throws SQLException { + insertPerson(6, "John", "Doe", 2, 2); + + IgniteException e = (IgniteException)GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + doBatchedInsert(); + + return null; + } + }, IgniteException.class, "Duplicate key during INSERT [key=KeyCacheObjectImpl " + + "[part=6, val=6, hasValBytes=true]]"); + + assertTrue(e.getCause() instanceof BatchUpdateException); + + assertTrue(e.getCause().getMessage().contains("Duplicate key during INSERT [key=KeyCacheObjectImpl " + + "[part=6, val=6, hasValBytes=true]]")); + + // First we insert id 7, then 6. Still, 7 is not in the cache as long as the whole batch has failed inside tx. + assertEquals(Collections.emptyList(), execute("SELECT * FROM \"Person\".Person where id > 6 order by id")); + } + + /** + * + */ + private void doBatchedInsert() throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + try { + try (PreparedStatement ps = conn.prepareStatement("INSERT INTO \"Person\".person " + + "(id, firstName, lastName, cityId, companyId) values (?, ?, ?, ?, ?)")) { + ps.setInt(1, 7); + + ps.setString(2, "Mary"); + + ps.setString(3, "Lee"); + + ps.setInt(4, 1); + + ps.setInt(5, 3); + + ps.addBatch(); + + ps.setInt(1, 6); + + ps.setString(2, "John"); + + ps.setString(3, "Doe"); + + ps.setInt(4, 2); + + ps.setInt(5, 2); + + ps.addBatch(); + + ps.executeBatch(); + } + } + catch (SQLException e) { + throw new IgniteException(e); + } + } + }); + } + + /** + * + */ + public void testInsertAndQueryMultipleCaches() throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + insertCity(conn, 5, "St Petersburg", 6000); + + insertCompany(conn, 6, "VK", 5); + + insertPerson(conn, 6, "Peter", "Sergeev", 5, 6); + } + }); + + try (Connection c = connect("distributedJoins=true")) { + assertEquals(l(l(5, "St Petersburg", 6000, 6, 5, "VK", 6, "Peter", "Sergeev", 5, 6)), + execute(c, "SELECT * FROM City left join Company on City.id = Company.\"cityid\" " + + "left join \"Person\".Person p on City.id = p.cityid WHERE p.id = 6 or company.id = 6")); + } + } + + /** + * + */ + public void testColocatedJoinSelectAndInsertInTransaction() throws SQLException { + // We'd like to put some Google into cities with over 1K population which don't have it yet + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + List ids = flat(execute(conn, "SELECT distinct City.id from City left join Company c on " + + "City.id = c.\"cityid\" where population >= 1000 and c.name <> 'Google' order by City.id")); + + assertEqualsCollections(l(1, 2), ids); + + int i = 5; + + for (int l : ids) + insertCompany(conn, ++i, "Google", l); + } + }); + + assertEqualsCollections(l("Los Angeles", "Seattle", "New York"), flat(execute("SELECT City.name from City " + + "left join Company c on city.id = c.\"cityid\" WHERE c.name = 'Google' order by City.id"))); + } + + /** + * + */ + public void testDistributedJoinSelectAndInsertInTransaction() throws SQLException { + try (Connection c = connect("distributedJoins=true")) { + // We'd like to put some Google into cities with over 1K population which don't have it yet + executeInTransaction(c, new TransactionClosure() { + @Override public void apply(Connection conn) { + List res = flat(execute(conn, "SELECT p.id,p.name,c.id from Company c left join Product p on " + + "c.id = p.companyid left join City on city.id = c.\"cityid\" WHERE c.name <> 'Microsoft' " + + "and population < 1000")); + + assertEqualsCollections(l(3, "Mac", 5), res); + + insertProduct(conn, 4, (String)res.get(1), 1); + } + }); + } + + try (Connection c = connect("distributedJoins=true")) { + assertEqualsCollections(l("Windows", "Mac"), flat(execute(c, "SELECT p.name from Company c left join " + + "Product p on c.id = p.companyid WHERE c.name = 'Microsoft' order by p.id"))); + } + } + + /** + * + */ + public void testInsertFromExpression() throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + execute(conn, "insert into city (id, name, population) values (? + 1, ?, ?)", + 8, "Moscow", 15000); + } + }); + } + + /** + * + */ + public void testAutoRollback() throws SQLException { + try (Connection c = connect()) { + begin(c); + + insertPerson(c, 6, "John", "Doe", 2, 2); + } + + // Connection has not hung on close and update has not been applied. + assertTrue(personCache().query(new SqlFieldsQuery("SELECT * FROM \"Person\".Person WHERE id = 6")) + .getAll().isEmpty()); + } + + /** + * + */ + public void testRepeatableReadWithConcurrentDelete() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where firstname = 'John'"); + } + }, null); + } + + /** + * + */ + public void testRepeatableReadWithConcurrentFastDelete() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where id = 1"); + } + }, null); + } + + /** + * + */ + public void testRepeatableReadWithConcurrentCacheRemove() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + personCache().remove(1); + } + }, null); + } + + /** + * + */ + public void testRepeatableReadAndDeleteWithConcurrentDelete() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where firstname = 'John'"); + } + }, afterReadDel); + } + + /** + * + */ + public void testRepeatableReadAndDeleteWithConcurrentFastDelete() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where id = 1"); + } + }, afterReadDel); + } + + /** + * + */ + public void testRepeatableReadAndDeleteWithConcurrentCacheRemove() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + personCache().remove(1); + } + }, afterReadDel); + } + + /** + * + */ + public void testRepeatableReadAndFastDeleteWithConcurrentDelete() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where firstname = 'John'"); + } + }, afterReadFastDel); + } + + /** + * + */ + public void testRepeatableReadAndFastDeleteWithConcurrentFastDelete() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where id = 1"); + } + }, afterReadFastDel); + } + + /** + * + */ + public void testRepeatableReadAndFastDeleteWithConcurrentCacheRemove() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + personCache().remove(1); + } + }, afterReadFastDel); + } + + /** + * + */ + public void testRepeatableReadAndDeleteWithConcurrentDeleteAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where firstname = 'John'"); + } + }, afterReadDelAndRollback); + } + + /** + * + */ + public void testRepeatableReadAndDeleteWithConcurrentFastDeleteAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where id = 1"); + } + }, afterReadDelAndRollback); + } + + /** + * + */ + public void testRepeatableReadAndDeleteWithConcurrentCacheRemoveAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + personCache().remove(1); + } + }, afterReadDelAndRollback); + } + + /** + * + */ + public void testRepeatableReadAndFastDeleteWithConcurrentDeleteAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where firstname = 'John'"); + } + }, afterReadFastDelAndRollback); + } + + /** + * + */ + public void testRepeatableReadAndFastDeleteWithConcurrentFastDeleteAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "DELETE FROM \"Person\".Person where id = 1"); + } + }, afterReadFastDelAndRollback); + } + + /** + * + */ + public void testRepeatableReadAndFastDeleteWithConcurrentCacheRemoveAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + personCache().remove(1); + } + }, afterReadFastDelAndRollback); + } + + /** + * + */ + public void testRepeatableReadWithConcurrentUpdate() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "UPDATE \"Person\".Person SET lastname = 'Fix' where firstname = 'John'"); + } + }, null); + } + + /** + * + */ + public void testRepeatableReadWithConcurrentCacheReplace() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + Person p = new Person(); + + p.id = 1; + p.firstName = "Luke"; + p.lastName = "Maxwell"; + + personCache().replace(1, p); + } + }, null); + } + + /** + * + */ + public void testRepeatableReadAndUpdateWithConcurrentUpdate() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "UPDATE \"Person\".Person SET lastname = 'Fix' where firstname = 'John'"); + } + }, afterReadUpdate); + } + + /** + * + */ + public void testRepeatableReadAndUpdateWithConcurrentCacheReplace() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + Person p = new Person(); + + p.id = 1; + p.firstName = "Luke"; + p.lastName = "Maxwell"; + + personCache().replace(1, p); + } + }, afterReadUpdate); + } + + /** + * + */ + public void testRepeatableReadAndUpdateWithConcurrentUpdateAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + execute(conn, "UPDATE \"Person\".Person SET lastname = 'Fix' where firstname = 'John'"); + } + }, afterReadUpdateAndRollback); + } + + /** + * + */ + public void testRepeatableReadAndUpdateWithConcurrentCacheReplaceAndRollback() throws Exception { + doTestRepeatableRead(new IgniteInClosure() { + @Override public void apply(Connection conn) { + Person p = new Person(); + + p.id = 1; + p.firstName = "Luke"; + p.lastName = "Maxwell"; + + personCache().replace(1, p); + } + }, afterReadUpdateAndRollback); + } + + /** + * Perform repeatable reads and concurrent changes. + * @param concurrentWriteClo Updating closure. + * @param afterReadClo Closure making write changes that should also be made inside repeatable read transaction + * (must yield an exception). + * @throws Exception if failed. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + private void doTestRepeatableRead(final IgniteInClosure concurrentWriteClo, + final IgniteInClosure afterReadClo) throws Exception { + final CountDownLatch repeatableReadLatch = new CountDownLatch(1); + + final CountDownLatch initLatch = new CountDownLatch(1); + + final IgniteInternalFuture readFut = multithreadedAsync(new Callable() { + @Override public Object call() throws Exception { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + List before = flat(execute(conn, "SELECT * from \"Person\".Person where id = 1")); + + assertEqualsCollections(l(1, "John", "Smith", 1, 1), before); + + initLatch.countDown(); + + try { + U.await(repeatableReadLatch); + } + catch (IgniteInterruptedCheckedException e) { + throw new IgniteException(e); + } + + List after = flat(execute(conn, "SELECT * from \"Person\".Person where id = 1")); + + assertEqualsCollections(before, after); + + if (afterReadClo != null) + afterReadClo.apply(conn); + } + }); + + return null; + } + }, 1); + + IgniteInternalFuture conModFut = multithreadedAsync(new Callable() { + @Override public Object call() throws Exception { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + try { + U.await(initLatch); + } + catch (IgniteInterruptedCheckedException e) { + throw new IgniteException(e); + } + + concurrentWriteClo.apply(conn); + + repeatableReadLatch.countDown(); + } + }); + + return null; + } + }, 1); + + conModFut.get(); + + if (afterReadClo != null) { + IgniteCheckedException ex = (IgniteCheckedException)GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + readFut.get(); + + return null; + } + }, IgniteCheckedException.class, "Mvcc version mismatch."); + + assertTrue(X.hasCause(ex, SQLException.class)); + + assertTrue(X.getCause(ex).getMessage().contains("Mvcc version mismatch.")); + } + else + readFut.get(); + } + + /** + * Create a new connection, a new transaction and run given closure in its scope. + * @param clo Closure. + * @throws SQLException if failed. + */ + private void executeInTransaction(TransactionClosure clo) throws SQLException { + try (Connection conn = connect()) { + executeInTransaction(conn, clo); + } + } + + /** + * Create a new transaction and run given closure in its scope. + * @param conn Connection. + * @param clo Closure. + * @throws SQLException if failed. + */ + private void executeInTransaction(Connection conn, TransactionClosure clo) throws SQLException { + begin(conn); + + clo.apply(conn); + + commit(conn); + } + + /** + * @return Auto commit strategy for this test. + */ + abstract boolean autoCommit(); + + /** + * @param c Connection to begin a transaction on. + */ + private void begin(Connection c) throws SQLException { + if (autoCommit()) + execute(c, "BEGIN"); + } + + /** + * @param c Connection to begin a transaction on. + */ + private void commit(Connection c) throws SQLException { + if (autoCommit()) + execute(c, "COMMIT"); + else + c.commit(); + } + + /** + * @param c Connection to rollback a transaction on. + */ + private void rollback(Connection c) { + try { + if (autoCommit()) + execute(c, "ROLLBACK"); + else + c.rollback(); + } + catch (SQLException e) { + throw new IgniteException(e); + } + } + + /** + * @param sql Statement. + * @param args Arguments. + * @return Result set. + * @throws SQLException if failed. + */ + List> execute(String sql, Object... args) throws SQLException { + try (Connection c = connect()) { + c.setAutoCommit(true); + + return execute(c, sql, args); + } + } + + /** + * @param sql Statement. + * @param args Arguments. + * @return Result set. + * @throws RuntimeException if failed. + */ + protected List> execute(Connection conn, String sql, Object... args) { + try { + return super.execute(conn, sql, args); + } + catch (SQLException e) { + throw new IgniteException(e); + } + } + + /** + * @return New connection to default node. + * @throws SQLException if failed. + */ + private Connection connect() throws SQLException { + return connect(null); + } + + /** + * @param params Connection parameters. + * @return New connection to default node. + * @throws SQLException if failed. + */ + private Connection connect(String params) throws SQLException { + Connection c = connect(node(), params); + + c.setAutoCommit(false); + + return c; + } + + /** + * @param node Node to connect to. + * @param params Connection parameters. + * @return Thin JDBC connection to specified node. + */ + protected Connection connect(IgniteEx node, String params) { + try { + return super.connect(node, params); + } + catch (SQLException e) { + throw new AssertionError(e); + } + } + + /** + * @return Default node to fire queries from. + */ + private IgniteEx node() { + return grid(nodeIndex()); + } + + /** + * @return {@link Person} cache. + */ + private IgniteCache personCache() { + return node().cache("Person"); + } + + /** + * @return Node index to fire queries from. + */ + abstract int nodeIndex(); + + /** + * @param id New person's id. + * @param firstName First name. + * @param lastName Second name. + * @param cityId City id. + * @param companyId Company id. + * @throws SQLException if failed. + */ + private void insertPerson(final int id, final String firstName, final String lastName, final int cityId, + final int companyId) throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + insertPerson(conn, id, firstName, lastName, cityId, companyId); + } + }); + } + + /** + * @param c Connection. + * @param id New person's id. + * @param firstName First name. + * @param lastName Second name. + * @param cityId City id. + * @param companyId Company id. + */ + private void insertPerson(Connection c, int id, String firstName, String lastName, int cityId, int companyId) { + execute(c, "INSERT INTO \"Person\".person (id, firstName, lastName, cityId, companyId) values (?, ?, ?, ?, ?)", + id, firstName, lastName, cityId, companyId); + } + + /** + * @param id New city's id. + * @param name City name. + * @param population Number of people. + * @throws SQLException if failed. + */ + private void insertCity(final int id, final String name, final int population) throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + insertCity(conn, id, name, population); + } + }); + } + + /** + * @param c Connection. + * @param id New city's id. + * @param name City name. + * @param population Number of people. + */ + private void insertCity(Connection c, int id, String name, int population) { + execute(c, "INSERT INTO city (id, name, population) values (?, ?, ?)", id, name, population); + } + + /** + * @param id New company's id. + * @param name Company name. + * @param cityId City id. + * @throws SQLException if failed. + */ + private void insertCompany(final int id, final String name, final int cityId) throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + insertCompany(conn, id, name, cityId); + } + }); + } + + /** + * @param c Connection. + * @param id New company's id. + * @param name Company name. + * @param cityId City id. + */ + private void insertCompany(Connection c, int id, String name, int cityId) { + execute(c, "INSERT INTO company (id, name, \"cityid\") values (?, ?, ?)", id, name, cityId); + } + + /** + * @param id New product's id. + * @param name Product name. + * @param companyId Company id.. + * @throws SQLException if failed. + */ + private void insertProduct(final int id, final String name, final int companyId) throws SQLException { + executeInTransaction(new TransactionClosure() { + @Override public void apply(Connection conn) { + insertProduct(conn, id, name, companyId); + } + }); + } + + /** + * @param c Connection. + * @param id New product's id. + * @param name Product name. + * @param companyId Company id.. + */ + private void insertProduct(Connection c, int id, String name, int companyId) { + execute(c, "INSERT INTO product (id, name, companyid) values (?, ?, ?)", id, name, companyId); + } + + /** + * Person class. + */ + private final static class Person { + /** */ + @QuerySqlField + public int id; + + /** */ + @QuerySqlField + public String firstName; + + /** */ + @QuerySqlField + public String lastName; + } + + /** + * Closure to be executed in scope of a transaction. + */ + private abstract class TransactionClosure implements IgniteInClosure { + // No-op. + } + + /** + * @return List of given arguments. + */ + private static List l(Object... args) { + return F.asList(args); + } + + /** + * Flatten rows. + * @param rows Rows. + * @return Rows as a single list. + */ + @SuppressWarnings("unchecked") + private static List flat(Collection> rows) { + return new ArrayList<>(F.flatCollections((Collection>)rows)); + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientAutoCommitComplexSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientAutoCommitComplexSelfTest.java new file mode 100644 index 0000000000000..d5b505a784a03 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientAutoCommitComplexSelfTest.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.jdbc.thin; + +/** + * + */ +public class JdbcThinTransactionsClientAutoCommitComplexSelfTest extends JdbcThinTransactionsAbstractComplexSelfTest { + /** {@inheritDoc} */ + @Override boolean autoCommit() { + return true; + } + + /** {@inheritDoc} */ + @Override int nodeIndex() { + return CLI_IDX; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.java new file mode 100644 index 0000000000000..7fa69fdc468a3 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.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.jdbc.thin; + +/** + * + */ +public class JdbcThinTransactionsClientNoAutoCommitComplexSelfTest extends JdbcThinTransactionsAbstractComplexSelfTest { + /** {@inheritDoc} */ + @Override boolean autoCommit() { + return false; + } + + /** {@inheritDoc} */ + @Override int nodeIndex() { + return CLI_IDX; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java new file mode 100644 index 0000000000000..1619996be1467 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java @@ -0,0 +1,447 @@ +/* + * 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.jdbc.thin; + +import java.sql.BatchUpdateException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.processors.query.NestedTxMode; +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.GridStringLogger; +import org.apache.ignite.testframework.GridTestUtils; +import org.jetbrains.annotations.NotNull; + +/** + * Tests to check behavior with transactions on. + */ +public class JdbcThinTransactionsSelfTest extends JdbcThinAbstractSelfTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String URL = "jdbc:ignite:thin://127.0.0.1"; + + /** Logger. */ + private GridStringLogger log; + + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setMvccEnabled(true); + + cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME)); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + + cfg.setMarshaller(new BinaryMarshaller()); + + cfg.setGridLogger(log = new GridStringLogger()); + + return cfg; + } + + /** + * @param name Cache name. + * @return Cache configuration. + * @throws Exception In case of error. + */ + private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception { + CacheConfiguration cfg = defaultCacheConfiguration(); + + cfg.setName(name); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + startGrid(0); + + try (Connection c = c(true, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + s.execute("CREATE TABLE INTS (k int primary key, v int) WITH \"cache_name=ints,wrap_value=false," + + "atomicity=transactional\""); + } + } + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @param autoCommit Auto commit mode. + * @param nestedTxMode Nested transactions mode. + * @return Connection. + * @throws SQLException if failed. + */ + private static Connection c(boolean autoCommit, NestedTxMode nestedTxMode) throws SQLException { + Connection res = DriverManager.getConnection(URL + "/?nestedTransactionsMode=" + nestedTxMode.name()); + + res.setAutoCommit(autoCommit); + + return res; + } + + /** + * + */ + public void testTransactionsBeginCommitRollback() throws IgniteCheckedException { + final AtomicBoolean stop = new AtomicBoolean(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + try { + try (Connection c = c(false, NestedTxMode.ERROR)) { + while (!stop.get()) { + try (Statement s = c.createStatement()) { + s.execute("BEGIN"); + + c.commit(); + + s.execute("BEGIN"); + + c.rollback(); + } + } + } + } + catch (SQLException e) { + throw new AssertionError(e); + } + } + }, 8, "jdbc-transactions"); + + U.sleep(5000); + + stop.set(true); + + fut.get(); + } + + /** + * + */ + public void testTransactionsBeginCommitRollbackAutocommit() throws IgniteCheckedException { + GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + try { + try (Connection c = c(true, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + s.execute("BEGIN"); + + s.execute("COMMIT"); + + s.execute("BEGIN"); + + s.execute("ROLLBACK"); + } + } + } + catch (SQLException e) { + throw new AssertionError(e); + } + } + }, 8, "jdbc-transactions").get(); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOff() throws SQLException { + try (Connection c = c(false, NestedTxMode.IGNORE)) { + doNestedTxStart(c, false); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOff() throws SQLException { + try (Connection c = c(false, NestedTxMode.COMMIT)) { + doNestedTxStart(c, false); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOff() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(false, NestedTxMode.ERROR)) { + doNestedTxStart(c, false); + } + + throw new AssertionError(); + } + }, SQLException.class, "Transaction has already been started."); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOn() throws SQLException { + try (Connection c = c(true, NestedTxMode.IGNORE)) { + doNestedTxStart(c, false); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOn() throws SQLException { + try (Connection c = c(true, NestedTxMode.COMMIT)) { + doNestedTxStart(c, false); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOn() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(true, NestedTxMode.ERROR)) { + doNestedTxStart(c, false); + } + + throw new AssertionError(); + } + }, SQLException.class, "Transaction has already been started."); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOffBatched() throws SQLException { + try (Connection c = c(false, NestedTxMode.IGNORE)) { + doNestedTxStart(c, true); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOffBatched() throws SQLException { + try (Connection c = c(false, NestedTxMode.COMMIT)) { + doNestedTxStart(c, true); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOffBatched() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(false, NestedTxMode.ERROR)) { + doNestedTxStart(c, true); + } + + throw new AssertionError(); + } + }, BatchUpdateException.class, "Transaction has already been started."); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOnBatched() throws SQLException { + try (Connection c = c(true, NestedTxMode.IGNORE)) { + doNestedTxStart(c, true); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOnBatched() throws SQLException { + try (Connection c = c(true, NestedTxMode.COMMIT)) { + doNestedTxStart(c, true); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOnBatched() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(true, NestedTxMode.ERROR)) { + doNestedTxStart(c, true); + } + + throw new AssertionError(); + } + }, BatchUpdateException.class, "Transaction has already been started."); + } + + /** + * Try to start nested transaction via batch as well as separate statements. + * @param conn Connection. + * @param batched Whether {@link Statement#executeBatch()} should be used. + * @throws SQLException if failed. + */ + private void doNestedTxStart(Connection conn, boolean batched) throws SQLException { + try (Statement s = conn.createStatement()) { + s.executeQuery("SELECT * FROM INTS"); + + if (batched) { + s.addBatch("BEGIN"); + + s.addBatch("BEGIN"); + + s.executeBatch(); + } + else { + s.execute("BEGIN"); + + s.execute("BEGIN"); + } + } + } + + /** + * @throws SQLException if failed. + */ + public void testAutoCommitSingle() throws SQLException { + doTestAutoCommit(false); + } + + /** + * @throws SQLException if failed. + */ + public void testAutoCommitBatched() throws SQLException { + doTestAutoCommit(true); + } + + /** + * @param batched Batch mode flag. + * @throws SQLException if failed. + */ + private void doTestAutoCommit(boolean batched) throws SQLException { + IgniteCache cache = grid(0).cache("ints"); + + try (Connection c = c(false, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + assertFalse(s.executeQuery("SELECT * from INTS").next()); + + if (batched) { + s.addBatch("INSERT INTO INTS(k, v) values(1, 1)"); + + s.executeBatch(); + } + else + s.execute("INSERT INTO INTS(k, v) values(1, 1)"); + + // We haven't committed anything yet - this check shows that autoCommit flag is in effect. + assertTrue(cache.query(new SqlFieldsQuery("SELECT * from INTS")).getAll().isEmpty()); + + // We should see own updates. + assertTrue(s.executeQuery("SELECT * from INTS").next()); + + c.commit(); + + c.setAutoCommit(true); + + assertEquals(1, cache.get(1)); + + assertTrue(s.executeQuery("SELECT * from INTS").next()); + } + } + } + + /** + * Test that exception in one of the statements does not kill connection worker altogether. + * @throws SQLException if failed. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testExceptionHandling() throws SQLException { + try (Connection c = c(true, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + s.execute("INSERT INTO INTS(k, v) values(1, 1)"); + + assertEquals(1, grid(0).cache("ints").get(1)); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + s.execute("INSERT INTO INTS(x, y) values(1, 1)"); + + return null; + } + }, SQLException.class, "Failed to parse query"); + + s.execute("INSERT INTO INTS(k, v) values(2, 2)"); + + assertEquals(2, grid(0).cache("ints").get(2)); + } + } + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerAutoCommitComplexSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerAutoCommitComplexSelfTest.java new file mode 100644 index 0000000000000..3c473ab872d5f --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerAutoCommitComplexSelfTest.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.jdbc.thin; + +/** + * + */ +public class JdbcThinTransactionsServerAutoCommitComplexSelfTest extends JdbcThinTransactionsAbstractComplexSelfTest { + /** {@inheritDoc} */ + @Override boolean autoCommit() { + return true; + } + + /** {@inheritDoc} */ + @Override int nodeIndex() { + return 0; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.java new file mode 100644 index 0000000000000..655d4c5c8f934 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.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.jdbc.thin; + +/** + * + */ +public class JdbcThinTransactionsServerNoAutoCommitComplexSelfTest extends JdbcThinTransactionsAbstractComplexSelfTest { + /** {@inheritDoc} */ + @Override boolean autoCommit() { + return false; + } + + /** {@inheritDoc} */ + @Override int nodeIndex() { + return 0; + } +} diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java index 99cf84946ab92..51365f39766ae 100644 --- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java +++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java @@ -38,6 +38,8 @@ import java.util.Set; import java.util.TreeSet; import java.util.UUID; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistRequest; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.GridCodegenConverter; import org.apache.ignite.internal.GridDirectCollection; @@ -168,7 +170,7 @@ public static void main(String[] args) throws Exception { // gen.generateAll(true); -// gen.generateAndWrite(GridChangeGlobalStateMessageResponse.class); + gen.generateAndWrite(GridNearTxQueryResultsEnlistRequest.class); // gen.generateAndWrite(GridNearAtomicUpdateRequest.class); @@ -235,6 +237,8 @@ public static void main(String[] args) throws Exception { // gen.generateAndWrite(GridCacheVersionEx.class); // gen.generateAndWrite(GridH2DmlRequest.class); // gen.generateAndWrite(GridH2DmlResponse.class); +// gen.generateAndWrite(GridNearTxEnlistRequest.class); +// gen.generateAndWrite(GridNearTxEnlistResponse.class); } /** 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 2e35f37077b88..fb3789d4df227 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 @@ -1019,6 +1019,7 @@ public CacheAtomicityMode getAtomicityMode() { * @param atomicityMode Cache atomicity mode. * @return {@code this} for chaining. */ + @SuppressWarnings("unchecked") public CacheConfiguration setAtomicityMode(CacheAtomicityMode atomicityMode) { this.atomicityMode = atomicityMode; diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index ab547091dc822..3060caa6ce3e7 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -214,6 +214,13 @@ public class IgniteConfiguration { /** Default timeout after which long query warning will be printed. */ public static final long DFLT_LONG_QRY_WARN_TIMEOUT = 3000; + /** Default size of MVCC vacuum thread pool. */ + public static final int DFLT_MVCC_VACUUM_THREAD_CNT = 2; + + /** Default time interval between vacuum process runs (ms). */ + public static final int DFLT_MVCC_VACUUM_TIME_INTERVAL = 5000; + + /** Optional local Ignite instance name. */ private String igniteInstanceName; @@ -487,6 +494,15 @@ public class IgniteConfiguration { /** Client connector configuration. */ private ClientConnectorConfiguration cliConnCfg = ClientListenerProcessor.DFLT_CLI_CFG; + /** Flag whether MVCC is enabled. */ + private boolean mvccEnabled; + + /** Size of MVCC vacuum thread pool. */ + private int mvccVacuumThreadCnt = DFLT_MVCC_VACUUM_THREAD_CNT; + + /** Time interval between vacuum process runs (ms). */ + private int mvccVacuumTimeInterval = DFLT_MVCC_VACUUM_TIME_INTERVAL; + /** User authentication enabled. */ private boolean authEnabled; @@ -535,6 +551,7 @@ public IgniteConfiguration(IgniteConfiguration cfg) { addrRslvr = cfg.getAddressResolver(); allResolversPassReq = cfg.isAllSegmentationResolversPassRequired(); atomicCfg = cfg.getAtomicConfiguration(); + authEnabled = cfg.isAuthenticationEnabled(); autoActivation = cfg.isAutoActivationEnabled(); binaryCfg = cfg.getBinaryConfiguration(); dsCfg = cfg.getDataStorageConfiguration(); @@ -578,6 +595,9 @@ public IgniteConfiguration(IgniteConfiguration cfg) { metricsLogFreq = cfg.getMetricsLogFrequency(); metricsUpdateFreq = cfg.getMetricsUpdateFrequency(); mgmtPoolSize = cfg.getManagementThreadPoolSize(); + mvccEnabled = cfg.isMvccEnabled(); + mvccVacuumThreadCnt = cfg.mvccVacuumThreadCnt; + mvccVacuumTimeInterval = cfg.mvccVacuumTimeInterval; netTimeout = cfg.getNetworkTimeout(); nodeId = cfg.getNodeId(); odbcCfg = cfg.getOdbcConfiguration(); @@ -612,7 +632,6 @@ public IgniteConfiguration(IgniteConfiguration cfg) { utilityCachePoolSize = cfg.getUtilityCacheThreadPoolSize(); waitForSegOnStart = cfg.isWaitForSegmentOnStart(); warmupClos = cfg.getWarmupClosure(); - authEnabled = cfg.isAuthenticationEnabled(); } /** @@ -2983,6 +3002,69 @@ public IgniteConfiguration setFailureHandler(FailureHandler failureHnd) { return cliConnCfg; } + /** + * Whether or not MVCC is enabled. + * + * @return {@code True} if MVCC is enabled. + */ + public boolean isMvccEnabled() { + return mvccEnabled; + } + + /** + * Sets MVCC enabled flag. + * + * @param mvccEnabled MVCC enabled flag. + * @return {@code this} for chaining. + */ + public IgniteConfiguration setMvccEnabled(boolean mvccEnabled) { + this.mvccEnabled = mvccEnabled; + + return this; + } + + /** + * Returns number of MVCC vacuum cleanup threads. + * + * @return Number of MVCC vacuum cleanup threads. + */ + public int getMvccVacuumThreadCnt() { + return mvccVacuumThreadCnt; + } + + /** + * Sets number of MVCC vacuum cleanup threads. + * + * @param mvccVacuumThreadCnt Number of MVCC vacuum cleanup threads. + * @return {@code this} for chaining. + */ + public IgniteConfiguration setMvccVacuumThreadCnt(int mvccVacuumThreadCnt) { + this.mvccVacuumThreadCnt = mvccVacuumThreadCnt; + + return this; + } + + /** + * Returns time interval between vacuum runs. + * + * @return Time interval between vacuum runs. + */ + public int getMvccVacuumTimeInterval() { + return mvccVacuumTimeInterval; + } + + /** + * Sets time interval between vacuum runs. + * + * @param mvccVacuumTimeInterval Time interval between vacuum runs. + * @return {@code this} for chaining. + */ + public IgniteConfiguration setMvccVacuumTimeInterval(int mvccVacuumTimeInterval) { + this.mvccVacuumTimeInterval = mvccVacuumTimeInterval; + + return this; + } + /** * Returns {@code true} if user authentication is enabled for cluster. Otherwise returns {@code false}. * Default value is false; authentication is disabled. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java index 504c83f34eb32..0cf3a6eb34771 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java @@ -64,7 +64,10 @@ enum DiscoveryDataExchangeType { QUERY_PROC, /** Authentication processor. */ - AUTH_PROC + AUTH_PROC, + + /** */ + CACHE_CRD_PROC } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index 051978c0fc728..0690565e04926 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager; import org.apache.ignite.internal.managers.indexing.GridIndexingManager; import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager; +import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor; import org.apache.ignite.internal.worker.WorkersRegistry; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; @@ -670,6 +671,11 @@ public interface GridKernalContext extends Iterable { */ public PlatformProcessor platform(); + /** + * @return Cache mvcc coordinator processor. + */ + public MvccProcessor coordinators(); + /** * @return PDS mode folder name resolver, also generates consistent ID in case new folder naming is used */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index 2be64e5d1330c..3b7b43072d99a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager; import org.apache.ignite.internal.managers.indexing.GridIndexingManager; import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager; +import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor; import org.apache.ignite.internal.worker.WorkersRegistry; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; @@ -290,6 +291,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude private DataStructuresProcessor dataStructuresProc; + /** Cache mvcc coordinators. */ + @GridToStringExclude + private MvccProcessor coordProc; + /** */ @GridToStringExclude private IgniteAuthenticationProcessor authProc; @@ -356,7 +361,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable /** */ @GridToStringExclude - Map customExecSvcs; + private Map customExecSvcs; /** */ @GridToStringExclude @@ -610,6 +615,8 @@ else if (comp instanceof PoolProcessor) poolProc = (PoolProcessor)comp; else if (comp instanceof GridMarshallerMappingProcessor) mappingProc = (GridMarshallerMappingProcessor)comp; + else if (comp instanceof MvccProcessor) + coordProc = (MvccProcessor)comp; else if (comp instanceof PdsFoldersResolver) pdsFolderRslvr = (PdsFoldersResolver)comp; else if (comp instanceof GridInternalSubscriptionProcessor) @@ -879,6 +886,11 @@ else if (helper instanceof HadoopHelper) return dataStructuresProc; } + /** {@inheritDoc} */ + @Override public MvccProcessor coordinators() { + return coordProc; + } + /** {@inheritDoc} */ @Override public IgniteAuthenticationProcessor authentication() { return authProc; 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 0b2d41a39ec89..68cea4f30e0f7 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 @@ -127,7 +127,10 @@ public enum GridTopic { TOPIC_AUTH, /** */ - TOPIC_EXCHANGE; + TOPIC_EXCHANGE, + + /** */ + TOPIC_CACHE_COORDINATOR; /** 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 eef2140d721c9..230c05c0a0890 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 @@ -125,6 +125,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; @@ -993,6 +994,7 @@ public void start( // be able to start receiving messages once discovery completes. try { startProcessor(new PdsConsistentIdProcessor(ctx)); + startProcessor(MvccUtils.createProcessor(ctx)); startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx)); startProcessor(new GridAffinityProcessor(ctx)); startProcessor(createComponent(GridSegmentationProcessor.class, ctx)); @@ -2761,7 +2763,7 @@ public IgniteInternalCache getCache(String name) { /** {@inheritDoc} */ @Override public IgniteCache createCache(CacheConfiguration cacheCfg) { A.notNull(cacheCfg, "cacheCfg"); - CU.validateCacheName(cacheCfg.getName()); + CU.validateNewCacheName(cacheCfg.getName()); guard(); @@ -2817,7 +2819,7 @@ public IgniteInternalCache getCache(String name) { /** {@inheritDoc} */ @Override public IgniteCache createCache(String cacheName) { - CU.validateCacheName(cacheName); + CU.validateNewCacheName(cacheName); guard(); @@ -2846,7 +2848,7 @@ public IgniteInternalCache getCache(String name) { @Override public IgniteBiTuple, Boolean> getOrCreateCache0( CacheConfiguration cacheCfg, boolean sql) { A.notNull(cacheCfg, "cacheCfg"); - CU.validateCacheName(cacheCfg.getName()); + CU.validateNewCacheName(cacheCfg.getName()); guard(); @@ -2912,7 +2914,7 @@ public IgniteInternalCache getCache(String name) { NearCacheConfiguration nearCfg ) { A.notNull(cacheCfg, "cacheCfg"); - CU.validateCacheName(cacheCfg.getName()); + CU.validateNewCacheName(cacheCfg.getName()); A.notNull(nearCfg, "nearCfg"); guard(); @@ -2941,7 +2943,7 @@ public IgniteInternalCache getCache(String name) { @Override public IgniteCache getOrCreateCache(CacheConfiguration cacheCfg, NearCacheConfiguration nearCfg) { A.notNull(cacheCfg, "cacheCfg"); - CU.validateCacheName(cacheCfg.getName()); + CU.validateNewCacheName(cacheCfg.getName()); A.notNull(nearCfg, "nearCfg"); guard(); @@ -2982,7 +2984,7 @@ public IgniteInternalCache getCache(String name) { /** {@inheritDoc} */ @Override public IgniteCache createNearCache(String cacheName, NearCacheConfiguration nearCfg) { - CU.validateCacheName(cacheName); + CU.validateNewCacheName(cacheName); A.notNull(nearCfg, "nearCfg"); guard(); @@ -3014,7 +3016,7 @@ public IgniteInternalCache getCache(String name) { /** {@inheritDoc} */ @Override public IgniteCache getOrCreateNearCache(String cacheName, NearCacheConfiguration nearCfg) { - CU.validateCacheName(cacheName); + CU.validateNewCacheName(cacheName); A.notNull(nearCfg, "nearCfg"); guard(); @@ -3143,7 +3145,7 @@ public IgniteInternalFuture destroyCachesAsync(Collection cacheNames, /** {@inheritDoc} */ @Override public IgniteCache getOrCreateCache(String cacheName) { - CU.validateCacheName(cacheName); + CU.validateNewCacheName(cacheName); guard(); @@ -3172,7 +3174,7 @@ public IgniteInternalFuture destroyCachesAsync(Collection cacheNames, */ public IgniteInternalFuture getOrCreateCacheAsync(String cacheName, String templateName, CacheConfigurationOverride cfgOverride, boolean checkThreadTx) { - CU.validateCacheName(cacheName); + CU.validateNewCacheName(cacheName); guard(); @@ -3192,7 +3194,7 @@ public IgniteInternalFuture getOrCreateCacheAsync(String cacheName, String te /** {@inheritDoc} */ @Override public void addCacheConfiguration(CacheConfiguration cacheCfg) { A.notNull(cacheCfg, "cacheCfg"); - CU.validateCacheName(cacheCfg.getName()); + CU.validateNewCacheName(cacheCfg.getName()); guard(); 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 ed16a7715b5b9..61a98a138b6ab 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 @@ -205,6 +205,9 @@ public final class IgniteNodeAttributes { /** Internal attribute name constant. */ public static final String ATTR_DYNAMIC_CACHE_START_ROLLBACK_SUPPORTED = ATTR_PREFIX + ".dynamic.cache.start.rollback.supported"; + /** Mvcc enabled flag. */ + public static final String ATTR_MVCC_ENABLED = ATTR_PREFIX + ".mvcc.enabled"; + /** * Enforces singleton. */ 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 2b408df7ef379..148bd2157841a 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 @@ -2349,12 +2349,8 @@ public void initializeDefaultCacheConfiguration(IgniteConfiguration cfg) throws "like TcpDiscoverySpi)"); for (CacheConfiguration ccfg : userCaches) { - if (CU.isHadoopSystemCache(ccfg.getName())) - throw new IgniteCheckedException("Cache name cannot be \"" + CU.SYS_CACHE_HADOOP_MR + - "\" because it is reserved for internal purposes."); - - if (CU.isUtilityCache(ccfg.getName())) - throw new IgniteCheckedException("Cache name cannot be \"" + CU.UTILITY_CACHE_NAME + + if (CU.isReservedCacheName(ccfg.getName())) + throw new IgniteCheckedException("Cache name cannot be \"" + ccfg.getName() + "\" because it is reserved for internal purposes."); if (IgfsUtils.matchIgfsCacheName(ccfg.getName())) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java index 92355a07a155b..5e40bbfc79dda 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java @@ -163,6 +163,16 @@ public interface ConnectionProperties { */ public void setSkipReducerOnUpdate(boolean skipReducerOnUpdate); + /** + * @return Nested transactions handling strategy. + */ + public String nestedTxMode(); + + /** + * @param nestedTxMode Nested transactions handling strategy. + */ + public void nestedTxMode(String nestedTxMode); + /** * Gets SSL connection mode. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java index 9371a078a7370..51a3837788bc4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java @@ -27,6 +27,7 @@ import org.apache.ignite.configuration.ClientConnectorConfiguration; import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.util.HostAndPortRange; +import org.apache.ignite.internal.processors.query.NestedTxMode; import org.apache.ignite.internal.util.typedef.F; /** @@ -99,6 +100,26 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa private BooleanProperty skipReducerOnUpdate = new BooleanProperty( "skipReducerOnUpdate", "Enable execution update queries on ignite server nodes", false, false); + /** Nested transactions handling strategy. */ + private StringProperty nestedTxMode = new StringProperty( + "nestedTransactionsMode", "Way to handle nested transactions", NestedTxMode.ERROR.name(), + new String[] { NestedTxMode.COMMIT.name(), NestedTxMode.ERROR.name(), NestedTxMode.IGNORE.name() }, + false, new PropertyValidator() { + private static final long serialVersionUID = 0L; + + @Override public void validate(String mode) throws SQLException { + if (!F.isEmpty(mode)) { + try { + NestedTxMode.valueOf(mode.toUpperCase()); + } + catch (IllegalArgumentException e) { + throw new SQLException("Invalid nested transactions handling mode, allowed values: " + + Arrays.toString(nestedTxMode.choices), SqlStateCode.CLIENT_CONNECTION_FAILED); + } + } + } + }); + /** SSL: Use SSL connection to Ignite node. */ private StringProperty sslMode = new StringProperty("sslMode", "The SSL mode of the connection", SSL_MODE_DISABLE, @@ -165,7 +186,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa /** Properties array. */ private final ConnectionProperty [] propsArray = { distributedJoins, enforceJoinOrder, collocated, replicatedOnly, autoCloseServerCursor, - tcpNoDelay, lazy, socketSendBuffer, socketReceiveBuffer, skipReducerOnUpdate, + tcpNoDelay, lazy, socketSendBuffer, socketReceiveBuffer, skipReducerOnUpdate, nestedTxMode, sslMode, sslProtocol, sslKeyAlgorithm, sslClientCertificateKeyStoreUrl, sslClientCertificateKeyStorePassword, sslClientCertificateKeyStoreType, sslTrustCertificateKeyStoreUrl, sslTrustCertificateKeyStorePassword, sslTrustCertificateKeyStoreType, @@ -436,6 +457,16 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa this.sslFactory.setValue(sslFactory); } + /** {@inheritDoc} */ + @Override public String nestedTxMode() { + return nestedTxMode.value(); + } + + /** {@inheritDoc} */ + @Override public void nestedTxMode(String val) { + nestedTxMode.setValue(val); + } + /** {@inheritDoc} */ @Override public void setUsername(String name) { user.setValue(name); @@ -815,11 +846,11 @@ void init(Properties props) throws SQLException { SqlStateCode.CLIENT_CONNECTION_FAILED); } - checkChoices(strVal); - if (validator != null) validator.validate(strVal); + checkChoices(strVal); + props.remove(name); init(strVal); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java index b36b3194c1a78..547b1db8cd414 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java @@ -197,7 +197,7 @@ void executeNative(String sql, SqlCommand cmd) throws SQLException { } sendRequest(new JdbcQueryExecuteRequest(JdbcStatementType.ANY_STATEMENT_TYPE, - schema, 1, 1, sql, null)); + schema, 1, 1, autoCommit, sql, null)); streamState = new StreamState((SqlSetStreamingCommand)cmd); } @@ -328,19 +328,18 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency, @Override public void setAutoCommit(boolean autoCommit) throws SQLException { ensureNotClosed(); - this.autoCommit = autoCommit; + // Do nothing if resulting value doesn't actually change. + if (autoCommit != this.autoCommit) { + doCommit(); - if (!autoCommit) - LOG.warning("Transactions are not supported."); + this.autoCommit = autoCommit; + } } /** {@inheritDoc} */ @Override public boolean getAutoCommit() throws SQLException { ensureNotClosed(); - if (!autoCommit) - LOG.warning("Transactions are not supported."); - return autoCommit; } @@ -351,7 +350,7 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency, if (autoCommit) throw new SQLException("Transaction cannot be committed explicitly in auto-commit mode."); - LOG.warning("Transactions are not supported."); + doCommit(); } /** {@inheritDoc} */ @@ -359,9 +358,21 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency, ensureNotClosed(); if (autoCommit) - throw new SQLException("Transaction cannot rollback in auto-commit mode."); + throw new SQLException("Transaction cannot be rolled back explicitly in auto-commit mode."); - LOG.warning("Transactions are not supported."); + try (Statement s = createStatement()) { + s.execute("ROLLBACK"); + } + } + + /** + * Send to the server {@code COMMIT} command. + * @throws SQLException if failed. + */ + private void doCommit() throws SQLException { + try (Statement s = createStatement()) { + s.execute("COMMIT"); + } } /** {@inheritDoc} */ @@ -927,7 +938,7 @@ private void executeBatch(boolean lastBatch) throws SQLException { respSem.acquire(); sendRequestNotWaitResponse( - new JdbcOrderedBatchExecuteRequest(schema, streamBatch, lastBatch, order)); + new JdbcOrderedBatchExecuteRequest(schema, streamBatch, autoCommit, lastBatch, order)); streamBatch = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java index 6c128a491d51f..e24ecbb3186a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java @@ -46,6 +46,7 @@ import org.apache.ignite.internal.util.typedef.F; import static java.sql.Connection.TRANSACTION_NONE; +import static java.sql.Connection.TRANSACTION_REPEATABLE_READ; import static java.sql.ResultSet.CONCUR_READ_ONLY; import static java.sql.ResultSet.HOLD_CURSORS_OVER_COMMIT; import static java.sql.ResultSet.TYPE_FORWARD_ONLY; @@ -630,17 +631,19 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData { /** {@inheritDoc} */ @Override public int getDefaultTransactionIsolation() throws SQLException { - return TRANSACTION_NONE; + return conn.igniteVersion().greaterThanEqual(2, 5, 0) ? TRANSACTION_REPEATABLE_READ : + TRANSACTION_NONE; } /** {@inheritDoc} */ @Override public boolean supportsTransactions() throws SQLException { - return false; + return conn.igniteVersion().greaterThanEqual(2, 5, 0); } /** {@inheritDoc} */ @Override public boolean supportsTransactionIsolationLevel(int level) throws SQLException { - return false; + return conn.igniteVersion().greaterThanEqual(2, 5, 0) && + TRANSACTION_REPEATABLE_READ == level; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index 30e446f65e3bf..f0f7337db79e6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -46,6 +46,8 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResultInfo; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.sql.SqlKeyword; import org.apache.ignite.internal.sql.SqlParseException; @@ -208,7 +210,7 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg } JdbcResult res0 = conn.sendRequest(new JdbcQueryExecuteRequest(stmtType, schema, pageSize, - maxRows, sql, args == null ? null : args.toArray(new Object[args.size()]))); + maxRows, conn.getAutoCommit(), sql, args == null ? null : args.toArray(new Object[args.size()]))); assert res0 != null; @@ -646,7 +648,8 @@ void checkStatementEligibleForBatching(String sql) throws SQLException { throw new SQLException("Batch is empty."); try { - JdbcBatchExecuteResult res = conn.sendRequest(new JdbcBatchExecuteRequest(conn.getSchema(), batch, false)); + JdbcBatchExecuteResult res = conn.sendRequest(new JdbcBatchExecuteRequest(conn.getSchema(), batch, + conn.getAutoCommit(), false)); if (res.errorCode() != ClientListenerResponse.STATUS_SUCCESS) { throw new BatchUpdateException(res.errorMessage(), IgniteQueryErrorCode.codeToSqlState(res.errorCode()), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java index 6128d07f0afe0..2c3f321a9585e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java @@ -326,6 +326,7 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL writer.writeBoolean(connProps.isAutoCloseServerCursor()); writer.writeBoolean(connProps.isLazy()); writer.writeBoolean(connProps.isSkipReducerOnUpdate()); + writer.writeString(connProps.nestedTxMode()); if (!F.isEmpty(connProps.getUsername())) { assert ver.compareTo(VER_2_5_0) >= 0 : "Authentication is supported since 2.5"; @@ -374,8 +375,9 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL + ", url=" + connProps.getUrl() + ']', SqlStateCode.CONNECTION_REJECTED); } - if (VER_2_4_0.equals(srvProtocolVer) || VER_2_3_0.equals(srvProtocolVer) || - VER_2_1_5.equals(srvProtocolVer)) + if (VER_2_4_0.equals(srvProtocolVer) + || VER_2_3_0.equals(srvProtocolVer) + || VER_2_1_5.equals(srvProtocolVer)) handshake(srvProtocolVer); else if (VER_2_1_0.equals(srvProtocolVer)) handshake_2_1_0(); @@ -541,8 +543,8 @@ private static int guessCapacity(JdbcRequest req) { int cnt = !F.isEmpty(qrys) ? Math.min(MAX_BATCH_QRY_CNT, qrys.size()) : 0; - // One additional byte for last batch flag. - cap = cnt * DYNAMIC_SIZE_MSG_CAP + 1; + // One additional byte for autocommit and last batch flags. + cap = cnt * DYNAMIC_SIZE_MSG_CAP + 2; } else if (req instanceof JdbcQueryCloseRequest) cap = QUERY_CLOSE_MSG_SIZE; 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 8d9a70034aeb3..f515d579ceadf 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 @@ -67,6 +67,7 @@ import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccMessage; import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter; import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; @@ -101,6 +102,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.GridTopic.TOPIC_CACHE_COORDINATOR; 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; @@ -1112,6 +1114,17 @@ private void processRegularMessage( return; } + if (msg.topicOrdinal() == TOPIC_CACHE_COORDINATOR.ordinal()) { + MvccMessage msg0 = (MvccMessage)msg.message(); + + // see IGNITE-8609 + /*if (msg0.processedFromNioThread()) + c.run(); + else*/ + ctx.getStripedExecutorService().execute(-1, c); + + return; + } if (plc == GridIoPolicy.SYSTEM_POOL && msg.partition() != GridIoMessage.STRIPE_DISABLED_PART) { ctx.getStripedExecutorService().execute(msg.partition(), c); @@ -1648,6 +1661,9 @@ else if (async) if (e.getCause() instanceof ClusterTopologyCheckedException) throw (ClusterTopologyCheckedException)e.getCause(); + if (!ctx.discovery().alive(node)) + throw new ClusterTopologyCheckedException("Failed to send message, node left: " + node.id()); + throw new IgniteCheckedException("Failed to send message (node may have left the grid or " + "TCP connection cannot be established due to firewall issues) " + "[node=" + node + ", topic=" + topic + 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 581c32e4b1fb0..8dddd8ba2ece5 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 @@ -47,13 +47,13 @@ import org.apache.ignite.internal.processors.cache.CacheObjectByteArrayImpl; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridChangeGlobalStateMessageResponse; import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl; import org.apache.ignite.internal.processors.cache.WalStateAckMessage; import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage; import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse; @@ -68,11 +68,15 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryEnlistRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryEnlistResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryFirstEnlistRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponse; @@ -97,6 +101,7 @@ 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.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage; import org.apache.ignite.internal.processors.cache.distributed.near.CacheVersionedValue; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; @@ -108,7 +113,24 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistResponse; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearUnlockRequest; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotWithoutTxs; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionImpl; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestQueryCntr; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestQueryId; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTx; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTxAndQueryCntr; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTxAndQueryId; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccActiveQueriesMessage; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccFutureResponse; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccQuerySnapshotRequest; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccSnapshotResponse; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccTxSnapshotRequest; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccWaitTxsRequest; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; @@ -927,6 +949,116 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; + case 136: + msg = new MvccTxSnapshotRequest(); + + break; + + case 137: + msg = new MvccAckRequestTx(); + + break; + + case 138: + msg = new MvccFutureResponse(); + + break; + + case 139: + msg = new MvccQuerySnapshotRequest(); + + break; + + case 140: + msg = new MvccAckRequestQueryCntr(); + + break; + + case 141: + msg = new MvccSnapshotResponse(); + + break; + + case 142: + msg = new MvccWaitTxsRequest(); + + break; + + case 143: + msg = new GridCacheMvccEntryInfo(); + + break; + + case 144: + msg = new GridDhtTxQueryEnlistResponse(); + + break; + + case 145: + msg = new MvccAckRequestQueryId(); + + break; + + case 146: + msg = new MvccAckRequestTxAndQueryCntr(); + + break; + + case 147: + msg = new MvccAckRequestTxAndQueryId(); + + break; + + case 148: + msg = new MvccVersionImpl(); + + break; + + case 149: + msg = new MvccActiveQueriesMessage(); + + break; + + case 150: + msg = new MvccSnapshotWithoutTxs(); + + break; + + case 151: + msg = new GridNearTxQueryEnlistRequest(); + + break; + + case 152: + msg = new GridNearTxQueryEnlistResponse(); + + break; + + case 153: + msg = new GridNearTxQueryResultsEnlistRequest(); + + break; + + case 154: + msg = new GridNearTxQueryResultsEnlistResponse(); + + break; + + case 155: + msg = new GridDhtTxQueryEnlistRequest(); + + break; + + case 156: + msg = new GridDhtTxQueryFirstEnlistRequest(); + + break; + + case 157: + msg = new GridDhtPartitionsUpdateCountersMap(); + + break; + // [-3..119] [124..129] [-23..-27] [-36..-55]- this // [120..123] - DR // [-4..-22, -30..-35] - SQL 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 8cdcbf367330d..84bcab1288c98 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 @@ -26,6 +26,7 @@ import org.apache.ignite.cluster.BaselineNode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -108,10 +109,14 @@ public class DiscoCache { /** */ private final P1 aliveNodePred; + /** */ + private final MvccCoordinator mvccCrd; + /** * @param topVer Topology version. * @param state Current cluster state. * @param loc Local node. + * @param mvccCrd MVCC coordinator node. * @param rmtNodes Remote nodes. * @param allNodes All nodes. * @param srvNodes Server nodes. @@ -130,6 +135,7 @@ public class DiscoCache { AffinityTopologyVersion topVer, DiscoveryDataClusterState state, ClusterNode loc, + MvccCoordinator mvccCrd, List rmtNodes, List allNodes, List srvNodes, @@ -148,6 +154,7 @@ public class DiscoCache { this.topVer = topVer; this.state = state; this.loc = loc; + this.mvccCrd = mvccCrd; this.rmtNodes = rmtNodes; this.allNodes = allNodes; this.srvNodes = srvNodes; @@ -157,7 +164,7 @@ public class DiscoCache { this.allCacheNodes = allCacheNodes; this.cacheGrpAffNodes = cacheGrpAffNodes; this.nodeMap = nodeMap; - alives.addAll(alives0); + this.alives.addAll(alives0); this.minNodeVer = minNodeVer; this.minSrvNodeVer = minSrvNodeVer; this.nodeIdToConsIdx = nodeIdToConsIdx; @@ -176,6 +183,13 @@ public class DiscoCache { }; } + /** + * @return Mvcc coordinator node. + */ + @Nullable public MvccCoordinator mvccCoordinator() { + return mvccCrd; + } + /** * @return Topology version. */ @@ -461,6 +475,7 @@ public DiscoCache copy(AffinityTopologyVersion ver, @Nullable DiscoveryDataClust ver, state == null ? this.state : state, loc, + mvccCrd, rmtNodes, allNodes, srvNodes, 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 a13f31e1ac2d1..d19e08bb9fbaa 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 @@ -84,6 +84,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cluster.BaselineTopology; import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage; @@ -159,6 +160,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS; 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_MVCC_ENABLED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_OFFHEAP_SIZE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM; @@ -644,6 +646,8 @@ private void onDiscovery0( updateClientNodes(node.id()); } + ctx.coordinators().onDiscoveryEvent(type, topSnapshot, topVer); + boolean locJoinEvt = type == EVT_NODE_JOINED && node.id().equals(locNode.id()); ChangeGlobalStateFinishMessage stateFinishMsg = null; @@ -1204,6 +1208,8 @@ private void checkAttributes(Iterable nodes) throws IgniteCheckedEx Boolean locSrvcCompatibilityEnabled = locNode.attribute(ATTR_SERVICES_COMPATIBILITY_MODE); Boolean locSecurityCompatibilityEnabled = locNode.attribute(ATTR_SECURITY_COMPATIBILITY_MODE); + Boolean locMvccEnabled = locNode.attribute(ATTR_MVCC_ENABLED); + for (ClusterNode n : nodes) { int rmtJvmMajVer = nodeJavaMajorVersion(n); @@ -1301,6 +1307,17 @@ private void checkAttributes(Iterable nodes) throws IgniteCheckedEx ", locNodeId=" + locNode.id() + ", rmtNode=" + U.toShortString(n) + "]"); } + Boolean rmtMvccEnabled = n.attribute(ATTR_MVCC_ENABLED); + + if (!F.eq(locMvccEnabled, rmtMvccEnabled)) { + throw new IgniteCheckedException("Remote node has MVCC mode different from local " + + "[locId8=" + U.id8(locNode.id()) + + ", locMvccMode=" + (Boolean.TRUE.equals(locMvccEnabled) ? "ENABLED" : "DISABLED") + + ", rmtId8=" + U.id8(n.id()) + + ", rmtMvccMode=" + (Boolean.TRUE.equals(rmtMvccEnabled) ? "ENABLED" : "DISABLED") + + ", rmtAddrs=" + U.addressesAsString(n) + ", rmtNode=" + U.toShortString(n) + "]"); + } + if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0 && ctx.security().enabled() // Matters only if security enabled. ) { @@ -2353,6 +2370,8 @@ public void reconnect() { Collection topSnapshot) { assert topSnapshot.contains(loc); + MvccCoordinator mvccCrd = ctx.coordinators().coordinatorFromDiscoveryEvent(); + HashSet alives = U.newHashSet(topSnapshot.size()); HashMap nodeMap = U.newHashMap(topSnapshot.size()); @@ -2454,6 +2473,7 @@ else if (node.version().compareTo(minVer) < 0) topVer, state, loc, + mvccCrd, Collections.unmodifiableList(rmtNodes), Collections.unmodifiableList(allNodes), Collections.unmodifiableList(srvNodes), @@ -3362,6 +3382,7 @@ public DiscoCache createDiscoCacheOnCacheChange( topVer, discoCache.state(), discoCache.localNode(), + discoCache.mvccCoordinator(), discoCache.remoteNodes(), allNodes, discoCache.serverNodes(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java index 5475bef76ebaa..ff0c66aa0d8ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheSharedManager; import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.AllocatedPageTracker; import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; /** @@ -42,6 +43,18 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh */ public void finishRecover() throws IgniteCheckedException; + /** + * Initializes disk store structures. + * + * @param cacheId Cache id. + * @param partitions Partitions count. + * @param workingDir Working directory. + * @param tracker Allocation tracker. + * @throws IgniteCheckedException If failed. + */ + void initialize(int cacheId, int partitions, String workingDir, AllocatedPageTracker tracker) + throws IgniteCheckedException; + /** * Callback called when a cache is starting. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java index 585336ace3c81..a555aaef4f245 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java @@ -184,7 +184,16 @@ public enum RecordType { RESERVED, /** Rotated id part record. */ - ROTATED_ID_PART_RECORD; + ROTATED_ID_PART_RECORD, + + /** */ + MVCC_DATA_PAGE_MARK_UPDATED_RECORD, + + /** */ + MVCC_DATA_PAGE_TX_STATE_HINT_UPDATED_RECORD, + + /** */ + MVCC_DATA_PAGE_NEW_TX_STATE_HINT_UPDATED_RECORD; /** */ private static final RecordType[] VALS = RecordType.values(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccMarkUpdatedRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccMarkUpdatedRecord.java new file mode 100644 index 0000000000000..5e89f8e960333 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccMarkUpdatedRecord.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.pagemem.wal.record.delta; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * MVCC mark updated record. + */ +public class DataPageMvccMarkUpdatedRecord extends PageDeltaRecord { + /** */ + private int itemId; + + /** */ + private long newMvccCrd; + + /** */ + private long newMvccCntr; + + /** */ + private int newMvccOpCntr; + + /** + * @param grpId Cache group ID. + * @param pageId Page ID. + * @param itemId Item id. + * @param newMvccCrd New MVCC coordinator version. + * @param newMvccCntr New MVCC counter version. + * @param newMvccOpCntr New MVCC operation counter. + */ + public DataPageMvccMarkUpdatedRecord(int grpId, long pageId, int itemId, long newMvccCrd, long newMvccCntr, int newMvccOpCntr) { + super(grpId, pageId); + + this.itemId = itemId; + this.newMvccCrd = newMvccCrd; + this.newMvccCntr = newMvccCntr; + this.newMvccOpCntr = newMvccOpCntr; + } + + /** {@inheritDoc} */ + @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException { + DataPageIO io = PageIO.getPageIO(pageAddr); + + io.updateNewVersion(pageAddr, itemId, pageMem.pageSize(), newMvccCrd, newMvccCntr, newMvccOpCntr); + } + + /** {@inheritDoc} */ + @Override public RecordType type() { + return RecordType.MVCC_DATA_PAGE_MARK_UPDATED_RECORD; + } + + /** + * @return Item id. + */ + public int itemId() { + return itemId; + } + + /** + * @return New MVCC coordinator version. + */ + public long newMvccCrd() { + return newMvccCrd; + } + + /** + * @return New MVCC counter version. + */ + public long newMvccCntr() { + return newMvccCntr; + } + + /** + * @return New MVCC operation counter. + */ + public int newMvccOpCntr() { + return newMvccOpCntr; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DataPageMvccMarkUpdatedRecord.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateNewTxStateHintRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateNewTxStateHintRecord.java new file mode 100644 index 0000000000000..4a244a1f2fe3d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateNewTxStateHintRecord.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.pagemem.wal.record.delta; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * MVCC update tx state hint record. + */ +public class DataPageMvccUpdateNewTxStateHintRecord extends PageDeltaRecord { + /** */ + private int itemId; + + /** */ + private byte txState; + + /** + * @param grpId Cache group ID. + * @param pageId Page ID. + * @param itemId Item id. + * @param txState Tx state hint. + */ + public DataPageMvccUpdateNewTxStateHintRecord(int grpId, long pageId, int itemId, byte txState) { + super(grpId, pageId); + + this.itemId = itemId; + this.txState = txState; + } + + /** {@inheritDoc} */ + @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException { + DataPageIO io = PageIO.getPageIO(pageAddr); + + io.updateNewTxState(pageAddr, itemId, pageMem.pageSize(), txState); + } + + /** {@inheritDoc} */ + @Override public RecordType type() { + return RecordType.MVCC_DATA_PAGE_NEW_TX_STATE_HINT_UPDATED_RECORD; + } + + /** + * @return Item id. + */ + public int itemId() { + return itemId; + } + + /** + * @return Tx state hint. + */ + public byte txState() { + return txState; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DataPageMvccUpdateNewTxStateHintRecord.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateTxStateHintRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateTxStateHintRecord.java new file mode 100644 index 0000000000000..7e53609064c06 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageMvccUpdateTxStateHintRecord.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.pagemem.wal.record.delta; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * MVCC update tx state hint record. + */ +public class DataPageMvccUpdateTxStateHintRecord extends PageDeltaRecord { + /** */ + private int itemId; + + /** */ + private byte txState; + + /** + * @param grpId Cache group ID. + * @param pageId Page ID. + * @param itemId Item id. + * @param txState Tx state hint. + */ + public DataPageMvccUpdateTxStateHintRecord(int grpId, long pageId, int itemId, byte txState) { + super(grpId, pageId); + + this.itemId = itemId; + this.txState = txState; + } + + /** {@inheritDoc} */ + @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException { + DataPageIO io = PageIO.getPageIO(pageAddr); + + io.updateTxState(pageAddr, itemId, pageMem.pageSize(), txState); + } + + /** {@inheritDoc} */ + @Override public RecordType type() { + return RecordType.MVCC_DATA_PAGE_TX_STATE_HINT_UPDATED_RECORD; + } + + /** + * @return Item id. + */ + public int itemId() { + return itemId; + } + + /** + * @return Tx state hint. + */ + public byte txState() { + return txState; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DataPageMvccUpdateTxStateHintRecord.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java index f78ab603a1232..1c321d3020b99 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java @@ -17,12 +17,12 @@ package org.apache.ignite.internal.processors.affinity; -import org.apache.ignite.cluster.ClusterNode; - import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.UUID; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; /** * Cached affinity calculations. @@ -90,4 +90,9 @@ public interface AffinityAssignment { * @return Backup partitions for specified node ID. */ public Set backupPartitions(UUID nodeId); + + /** + * @return Mvcc coordinator. + */ + public MvccCoordinator mvccCoordinator(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java index cbec1a1852eba..f96bc9d0c6fab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -40,6 +41,9 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable /** Topology version. */ private final AffinityTopologyVersion topVer; + /** */ + private final MvccCoordinator mvccCrd; + /** Collection of calculated affinity nodes. */ private List> assignment; @@ -73,6 +77,7 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable this.topVer = topVer; primary = new HashMap<>(); backup = new HashMap<>(); + mvccCrd = null; clientEvtChange = false; } @@ -83,7 +88,8 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable */ GridAffinityAssignment(AffinityTopologyVersion topVer, List> assignment, - List> idealAssignment) { + List> idealAssignment, + MvccCoordinator mvccCrd) { assert topVer != null; assert assignment != null; assert idealAssignment != null; @@ -91,6 +97,7 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable this.topVer = topVer; this.assignment = assignment; this.idealAssignment = idealAssignment.equals(assignment) ? assignment : idealAssignment; + this.mvccCrd = mvccCrd; primary = new HashMap<>(); backup = new HashMap<>(); @@ -110,6 +117,7 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable idealAssignment = aff.idealAssignment; primary = aff.primary; backup = aff.backup; + mvccCrd = aff.mvccCrd; clientEvtChange = true; } @@ -282,6 +290,11 @@ private void initPrimaryBackupMaps() { } } + /** {@inheritDoc} */ + @Override public MvccCoordinator mvccCoordinator() { + return mvccCrd; + } + /** {@inheritDoc} */ @Override public int hashCode() { return topVer.hashCode(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java index 34e2b0ad19b1d..cc2c17c3f3eda 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java @@ -43,6 +43,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.NodeOrderComparator; import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvents; import org.apache.ignite.internal.processors.cluster.BaselineTopology; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -202,11 +203,25 @@ public int groupId() { * @param affAssignment Affinity assignment for topology version. */ public void initialize(AffinityTopologyVersion topVer, List> affAssignment) { + MvccCoordinator mvccCrd = ctx.cache().context().coordinators().currentCoordinator(topVer); + + initialize(topVer, affAssignment, mvccCrd); + } + + /** + * Initializes affinity with given topology version and assignment. + * + * @param topVer Topology version. + * @param affAssignment Affinity assignment for topology version. + * @param mvccCrd Mvcc coordinator. + */ + public void initialize(AffinityTopologyVersion topVer, List> affAssignment, MvccCoordinator mvccCrd) { assert topVer.compareTo(lastVersion()) >= 0 : "[topVer = " + topVer + ", last=" + lastVersion() + ']'; assert idealAssignment != null; + assert mvccCrd == null || topVer.compareTo(mvccCrd.topologyVersion()) >= 0 : "[mvccCrd=" + mvccCrd + ", topVer=" + topVer + ']'; - GridAffinityAssignment assignment = new GridAffinityAssignment(topVer, affAssignment, idealAssignment); + GridAffinityAssignment assignment = new GridAffinityAssignment(topVer, affAssignment, idealAssignment, mvccCrd); HistoryAffinityAssignment hAff = affCache.put(topVer, new HistoryAffinityAssignment(assignment)); @@ -745,7 +760,9 @@ public void init(GridAffinityAssignmentCache aff) { idealAssignment(aff.idealAssignment()); - initialize(aff.lastVersion(), aff.assignments(aff.lastVersion())); + AffinityAssignment assign = aff.cachedAffinity(aff.lastVersion()); + + initialize(aff.lastVersion(), assign.assignment(), assign.mvccCoordinator()); } /** 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 08333c33e131a..4a0908c6071f6 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 @@ -423,7 +423,7 @@ private Map> keysToNodes(@Nullable final String c try { GridAffinityAssignment assign = assign0 instanceof GridAffinityAssignment ? (GridAffinityAssignment)assign0 : - new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment()); + new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment(), assign0.mvccCoordinator()); AffinityInfo info = new AffinityInfo( cctx.config().getAffinity(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java index abd5292799958..15d7e4e437ea7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java @@ -184,7 +184,7 @@ public AffinityJob() { GridAffinityAssignment assign = assign0 instanceof GridAffinityAssignment ? (GridAffinityAssignment)assign0 : - new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment()); + new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment(), assign0.mvccCoordinator()); return F.t( affinityMessage(ctx, cctx.config().getAffinity()), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java index 94eaab4bf54b1..1a6b2efea9006 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.affinity; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -44,16 +45,25 @@ public class HistoryAffinityAssignment implements AffinityAssignment { /** */ private final boolean clientEvtChange; + /** */ + private final MvccCoordinator mvccCrd; + /** * @param assign Assignment. */ - public HistoryAffinityAssignment(GridAffinityAssignment assign) { + HistoryAffinityAssignment(GridAffinityAssignment assign) { this.topVer = assign.topologyVersion(); this.assignment = assign.assignment(); this.idealAssignment = assign.idealAssignment(); + this.mvccCrd = assign.mvccCoordinator(); this.clientEvtChange = assign.clientEventChange(); } + /** {@inheritDoc} */ + @Override public MvccCoordinator mvccCoordinator() { + return mvccCrd; + } + /** {@inheritDoc} */ @Override public boolean clientEventChange() { return clientEvtChange; 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 0b448e145c370..5859452e3c23a 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 @@ -58,6 +58,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; 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.mvcc.MvccCoordinator; +import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.GridPartitionStateMap; @@ -458,7 +460,11 @@ void onCacheGroupCreated(CacheGroupContext grp) { if (grpHolder.client()) { ClientCacheDhtTopologyFuture topFut = new ClientCacheDhtTopologyFuture(topVer); - grp.topology().updateTopologyVersion(topFut, discoCache, -1, false); + grp.topology().updateTopologyVersion(topFut, + discoCache, + cctx.coordinators().currentCoordinator(), + -1, + false); grpHolder = new CacheGroupHolder1(grp, grpHolder.affinity()); @@ -506,6 +512,7 @@ else if (!fetchFuts.containsKey(grp.groupId())) { assert grp != null; GridDhtAffinityAssignmentResponse res = fetchAffinity(topVer, + cctx.coordinators().currentCoordinator(), null, discoCache, grp.affinity(), @@ -528,7 +535,11 @@ else if (!fetchFuts.containsKey(grp.groupId())) { new ClusterTopologyServerNotFoundException("All server nodes left grid.")); } - grp.topology().updateTopologyVersion(topFut, discoCache, -1, false); + grp.topology().updateTopologyVersion(topFut, + discoCache, + cctx.coordinators().currentCoordinator(), + -1, + false); grp.topology().update(topVer, partMap, null, Collections.emptySet(), null, null); @@ -1284,7 +1295,12 @@ private void initAffinity(CacheGroupDescriptor desc, fetchFut.init(false); - fetchAffinity(evts.topologyVersion(), evts, evts.discoveryCache(), aff, fetchFut); + fetchAffinity(evts.topologyVersion(), + cctx.coordinators().currentCoordinator(), + evts, + evts.discoveryCache(), + aff, + fetchFut); } } @@ -1682,6 +1698,7 @@ private void fetchAffinityOnJoin(GridDhtPartitionsExchangeFuture fut) throws Ign int grpId = fetchFut.groupId(); fetchAffinity(topVer, + cctx.coordinators().currentCoordinator(), fut.events(), fut.events().discoveryCache(), cctx.cache().cacheGroup(grpId).affinity(), @@ -1691,6 +1708,7 @@ private void fetchAffinityOnJoin(GridDhtPartitionsExchangeFuture fut) throws Ign /** * @param topVer Topology version. + * @param mvccCrd Mvcc coordinator to set in affinity. * @param events Discovery events. * @param discoCache Discovery data cache. * @param affCache Affinity. @@ -1698,7 +1716,9 @@ private void fetchAffinityOnJoin(GridDhtPartitionsExchangeFuture fut) throws Ign * @throws IgniteCheckedException If failed. * @return Affinity assignment response. */ - private GridDhtAffinityAssignmentResponse fetchAffinity(AffinityTopologyVersion topVer, + private GridDhtAffinityAssignmentResponse fetchAffinity( + AffinityTopologyVersion topVer, + MvccCoordinator mvccCrd, @Nullable ExchangeDiscoveryEvents events, DiscoCache discoCache, GridAffinityAssignmentCache affCache, @@ -1711,7 +1731,7 @@ private GridDhtAffinityAssignmentResponse fetchAffinity(AffinityTopologyVersion if (res == null) { List> aff = affCache.calculate(topVer, events, discoCache); - affCache.initialize(topVer, aff); + affCache.initialize(topVer, aff, mvccCrd); } else { List> idealAff = res.idealAffinityAssignment(discoCache); @@ -1728,7 +1748,7 @@ private GridDhtAffinityAssignmentResponse fetchAffinity(AffinityTopologyVersion assert aff != null : res; - affCache.initialize(topVer, aff); + affCache.initialize(topVer, aff, mvccCrd); } return res; @@ -1847,6 +1867,7 @@ public IgniteInternalFuture initCoordinatorCaches( @Override public void applyx(IgniteInternalFuture fetchFut) throws IgniteCheckedException { fetchAffinity(prev.topologyVersion(), + null, // Pass null mvcc coordinator, this affinity version should be used for queries. prev.events(), prev.events().discoveryCache(), aff, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java index 49f77fa808dba..614d7c06fbf56 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java @@ -37,6 +37,18 @@ public class CacheEntryInfoCollection implements Message { @GridDirectCollection(GridCacheEntryInfo.class) private List infos; + /** */ + public CacheEntryInfoCollection() { + // No-op + } + + /** + * @param infos List of cache entry info. + */ + public CacheEntryInfoCollection(List infos) { + this.infos = infos; + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java index c8fe283fc115d..c100d161ff9a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java @@ -30,6 +30,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataPageEvictionMode; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.TopologyValidator; import org.apache.ignite.events.CacheRebalancingEvent; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; @@ -60,6 +61,7 @@ import org.apache.ignite.mxbean.CacheGroupMetricsMXBean; 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.REPLICATED; import static org.apache.ignite.cache.CacheRebalanceMode.NONE; @@ -147,6 +149,9 @@ public class CacheGroupContext { /** */ private boolean qryEnabled; + /** */ + private boolean mvccEnabled; + /** MXBean. */ private CacheGroupMetricsMXBean mxBean; @@ -218,6 +223,28 @@ public class CacheGroupContext { caches = new ArrayList<>(); mxBean = new CacheGroupMetricsMXBeanImpl(this); + + mvccEnabled = mvccEnabled(ctx.gridConfig(), ccfg, cacheType); + } + + /** + * @param cfg Ignite configuration. + * @param ccfg Cache configuration. + * @param cacheType Cache typr. + * @return {@code True} if mvcc is enabled for given cache. + */ + public static boolean mvccEnabled(IgniteConfiguration cfg, CacheConfiguration ccfg, CacheType cacheType) { + return cfg.isMvccEnabled() && + cacheType == CacheType.USER && + ccfg.getCacheMode() != LOCAL && + ccfg.getAtomicityMode() == TRANSACTIONAL; + } + + /** + * @return Mvcc flag. + */ + public boolean mvccEnabled() { + return mvccEnabled; } /** @@ -395,6 +422,13 @@ public boolean eventRecordable(int type) { return cacheType.userCache() && ctx.gridEvents().isRecordable(type); } + /** + * @return {@code True} if cache created by user. + */ + public boolean userCache() { + return cacheType.userCache(); + } + /** * Adds rebalancing event. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOperationContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOperationContext.java index 8a7afe7175e46..53f9e229363a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOperationContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOperationContext.java @@ -163,7 +163,7 @@ public CacheOperationContext keepBinary() { /** * Gets data center ID. * - * @return Client ID. + * @return Datacenter ID. */ @Nullable public Byte dataCenterId() { return dataCenterId; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 3aaf7f3c08c78..8eab9c01521b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -147,7 +147,7 @@ public void onStart(CacheJoinNodeDiscoveryData joinDiscoData) throws IgniteCheck if (ccfg == null) grpCfgs.put(info.cacheData().config().getGroupName(), info.cacheData().config()); else - validateCacheGroupConfiguration(ccfg, info.cacheData().config()); + validateCacheGroupConfiguration(ccfg, info.cacheData().config(), info.cacheType()); } String conflictErr = processJoiningNode(joinDiscoData, ctx.localNodeId(), true); @@ -220,7 +220,7 @@ public void onKernalStart(boolean checkConsistency) throws IgniteCheckedExceptio } if (checkConsistency) - validateStartCacheConfiguration(locCfg); + validateStartCacheConfiguration(locCfg, cacheData.cacheType()); } } @@ -1864,16 +1864,17 @@ else if (exchActions == null) { /** * @param ccfg Cache configuration to start. + * @param cacheType Cache type. * @throws IgniteCheckedException If failed. */ - public void validateStartCacheConfiguration(CacheConfiguration ccfg) throws IgniteCheckedException { + void validateStartCacheConfiguration(CacheConfiguration ccfg, CacheType cacheType) throws IgniteCheckedException { if (ccfg.getGroupName() != null) { CacheGroupDescriptor grpDesc = cacheGroupByName(ccfg.getGroupName()); if (grpDesc != null) { assert ccfg.getGroupName().equals(grpDesc.groupName()); - validateCacheGroupConfiguration(grpDesc.config(), ccfg); + validateCacheGroupConfiguration(grpDesc.config(), ccfg, cacheType); } } } @@ -1881,9 +1882,10 @@ public void validateStartCacheConfiguration(CacheConfiguration ccfg) throws Igni /** * @param cfg Existing configuration. * @param startCfg Cache configuration to start. + * @param cacheType Cache type. * @throws IgniteCheckedException If validation failed. */ - private void validateCacheGroupConfiguration(CacheConfiguration cfg, CacheConfiguration startCfg) + private void validateCacheGroupConfiguration(CacheConfiguration cfg, CacheConfiguration startCfg, CacheType cacheType) throws IgniteCheckedException { GridCacheAttributes attr1 = new GridCacheAttributes(cfg); GridCacheAttributes attr2 = new GridCacheAttributes(startCfg); @@ -1891,6 +1893,11 @@ private void validateCacheGroupConfiguration(CacheConfiguration cfg, CacheConfig CU.validateCacheGroupsAttributesMismatch(log, cfg, startCfg, "cacheMode", "Cache mode", cfg.getCacheMode(), startCfg.getCacheMode(), true); + CU.validateCacheGroupsAttributesMismatch(log, cfg, startCfg, "mvccEnabled", "MVCC mode", + CacheGroupContext.mvccEnabled(ctx.config(), cfg, cacheType), + CacheGroupContext.mvccEnabled(ctx.config(), startCfg, cacheType), + true); + CU.validateCacheGroupsAttributesMismatch(log, cfg, startCfg, "affinity", "Affinity function", attr1.cacheAffinityClassName(), attr2.cacheAffinityClassName(), true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java index 4046c98b29414..34ed048542c16 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeContext.java @@ -17,11 +17,15 @@ package org.apache.ignite.internal.processors.cache; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; +import java.util.UUID; import org.apache.ignite.cluster.ClusterNode; 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.util.GridLongList; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -51,11 +55,20 @@ public class ExchangeContext { /** */ private final boolean compatibilityNode = getBoolean(IGNITE_EXCHANGE_COMPATIBILITY_VER_1, false); + /** */ + private final boolean newMvccCrd; + + /** Currently running mvcc queries, initialized when mvcc coordinator is changed. */ + private Map activeQueries; + /** * @param crd Coordinator flag. + * @param newMvccCrd {@code True} if new coordinator assigned during this exchange. * @param fut Exchange future. */ - public ExchangeContext(boolean crd, GridDhtPartitionsExchangeFuture fut) { + public ExchangeContext(boolean crd, boolean newMvccCrd, GridDhtPartitionsExchangeFuture fut) { + this.newMvccCrd = newMvccCrd; + int protocolVer = exchangeProtocolVersion(fut.firstEventCache().minimumNodeVersion()); if (compatibilityNode || (crd && fut.localJoinExchange())) { @@ -124,6 +137,34 @@ public boolean mergeExchanges() { return merge; } + /** + * @return {@code True} if new node assigned as mvcc coordinator node during this exchange. + */ + public boolean newMvccCoordinator() { + return newMvccCrd; + } + + /** + * @return Active queries. + */ + public Map activeQueries() { + return activeQueries; + } + + /** + * @param nodeId Node ID. + * @param nodeQueries Node queries. + */ + public void addActiveQueries(UUID nodeId, @Nullable GridLongList nodeQueries) { + if (nodeQueries == null) + return; + + if (activeQueries == null) + activeQueries = new HashMap<>(); + + activeQueries.put(nodeId, nodeQueries); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(ExchangeContext.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java index 96c5e29fb4aed..c99eb006002e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java @@ -49,6 +49,7 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.internal.AsyncSupportAdapter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.GridKernalState; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.future.IgniteFutureImpl; @@ -145,6 +146,8 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { CacheOperationGate opGate = onEnter(); try { + MvccUtils.verifyMvccOperationSupport(delegate.context(), "withExpiryPolicy"); + return new GatewayProtectedCacheProxy<>(delegate, opCtx.withExpiryPolicy(plc), lock); } finally { 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 bc14c548af92b..159d6810e0521 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 @@ -92,6 +92,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter; @@ -138,7 +140,6 @@ import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -816,6 +817,9 @@ public String toString() { ctx.checkSecurity(SecurityPermission.CACHE_READ); + //TODO IGNITE-7955 + MvccUtils.verifyMvccOperationSupport(ctx, "Peek"); + PeekModes modes = parsePeekModes(peekModes, false); KeyCacheObject cacheKey = ctx.toCacheKeyObject(key); @@ -1121,6 +1125,9 @@ public List> splitClearLocally(boolean srv, bool @Override public void clearLocally(boolean srv, boolean near, boolean readers) { ctx.checkSecurity(SecurityPermission.CACHE_REMOVE); + //TODO IGNITE-7952 + MvccUtils.verifyMvccOperationSupport(ctx, "Clear"); + List> jobs = splitClearLocally(srv, near, readers); if (!F.isEmpty(jobs)) { @@ -1190,6 +1197,9 @@ public List> splitClearLocally(boolean srv, bool * @throws IgniteCheckedException In case of error. */ private void clear(@Nullable Set keys) throws IgniteCheckedException { + //TODO IGNITE-7952 + MvccUtils.verifyMvccOperationSupport(ctx, "Clear"); + if (isLocal()) { if (keys == null) clearLocally(true, false, false); @@ -1207,6 +1217,9 @@ private void clear(@Nullable Set keys) throws IgniteCheckedExceptio * @return Future. */ private IgniteInternalFuture clearAsync(@Nullable final Set keys) { + //TODO IGNITE-7952 + MvccUtils.verifyMvccOperationSupport(ctx, "Clear"); + if (isLocal()) return clearLocallyAsync(keys); else @@ -1262,6 +1275,9 @@ public void clearLocally(Collection keys, boolean readers) { if (F.isEmpty(keys)) return; + //TODO IGNITE-7952 + MvccUtils.verifyMvccOperationSupport(ctx, "Clear"); + GridCacheVersion obsoleteVer = ctx.versions().next(); for (KeyCacheObject key : keys) { @@ -1881,7 +1897,8 @@ public final IgniteInternalFuture> getAllAsync(@Nullable final Collect skipVals, /*keep cache objects*/false, recovery, - needVer); + needVer, + null); // TODO IGNITE-7371 } /** @@ -1896,6 +1913,7 @@ public final IgniteInternalFuture> getAllAsync(@Nullable final Collect * @param skipVals Skip values flag. * @param keepCacheObjects Keep cache objects. * @param needVer If {@code true} returns values as tuples containing value and version. + * @param mvccSnapshot MVCC snapshot. * @return Future. */ protected final IgniteInternalFuture> getAllAsync0( @@ -1910,7 +1928,8 @@ protected final IgniteInternalFuture> getAllAsync0( final boolean skipVals, final boolean keepCacheObjects, final boolean recovery, - final boolean needVer + final boolean needVer, + MvccSnapshot mvccSnapshot ) { if (F.isEmpty(keys)) return new GridFinishedFuture<>(Collections.emptyMap()); @@ -1967,7 +1986,8 @@ protected final IgniteInternalFuture> getAllAsync0( boolean skipEntry = readNoEntry; if (readNoEntry) { - CacheDataRow row = ctx.offheap().read(ctx, key); + CacheDataRow row = mvccSnapshot != null ? ctx.offheap().mvccRead(ctx, key, mvccSnapshot) : + ctx.offheap().read(ctx, key); if (row != null) { long expireTime = row.expireTime(); @@ -2030,6 +2050,7 @@ else if (storeEnabled) taskName, expiry, !deserializeBinary, + mvccSnapshot, readerArgs); assert res != null; @@ -2054,6 +2075,7 @@ else if (storeEnabled) taskName, expiry, !deserializeBinary, + mvccSnapshot, readerArgs); if (res == null) @@ -3072,7 +3094,7 @@ protected IgniteInternalFuture getAndRemoveAsync0(final K key) { List keys = new ArrayList<>(Math.min(REMOVE_ALL_KEYS_BATCH, size())); do { - for (Iterator it = ctx.offheap().cacheIterator(ctx.cacheId(), true, true, null); + for (Iterator it = ctx.offheap().cacheIterator(ctx.cacheId(), true, true, null, null); it.hasNext() && keys.size() < REMOVE_ALL_KEYS_BATCH; ) keys.add((K)it.next().key()); @@ -3397,6 +3419,9 @@ public CacheMetricsImpl metrics0() { if (keyCheck) validateCacheKeys(keys); + //TODO IGNITE-7764 + MvccUtils.verifyMvccOperationSupport(ctx, "Lock"); + IgniteInternalFuture fut = lockAllAsync(keys, timeout); boolean isInterrupted = false; @@ -3425,6 +3450,9 @@ public CacheMetricsImpl metrics0() { if (keyCheck) validateCacheKey(key); + //TODO IGNITE-7764 + MvccUtils.verifyMvccOperationSupport(ctx, "Lock"); + return lockAllAsync(Collections.singletonList(key), timeout); } @@ -3534,6 +3562,9 @@ protected void checkJta() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void localLoadCache(final IgniteBiPredicate p, Object[] args) throws IgniteCheckedException { + //TODO IGNITE-7954 + MvccUtils.verifyMvccOperationSupport(ctx, "Load"); + final boolean replicate = ctx.isDrEnabled(); final AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion(); @@ -3671,6 +3702,9 @@ public IgniteInternalFuture loadAll( if (!ctx.store().configured()) return new GridFinishedFuture<>(); + //TODO IGNITE-7954 + MvccUtils.verifyMvccOperationSupport(ctx, "Load"); + CacheOperationContext opCtx = ctx.operationContextPerCall(); ExpiryPolicy plc = opCtx != null ? opCtx.expiry() : null; @@ -3755,6 +3789,9 @@ private void localLoadAndUpdate(final Collection keys) throws Ignit */ public void localLoad(Collection keys, @Nullable ExpiryPolicy plc, final boolean keepBinary) throws IgniteCheckedException { + //TODO IGNITE-7954 + MvccUtils.verifyMvccOperationSupport(ctx, "Load"); + final boolean replicate = ctx.isDrEnabled(); final AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion(); @@ -3827,6 +3864,9 @@ IgniteInternalFuture globalLoadCacheAsync(@Nullable IgniteBiPredicate p assert !F.isEmpty(nodes) : "There are not datanodes fo cache: " + ctx.name(); + //TODO IGNITE-7954 + MvccUtils.verifyMvccOperationSupport(ctx, "Load"); + final boolean keepBinary = opCtx != null && opCtx.isKeepBinary(); ComputeTaskInternalFuture fut = ctx.kernalContext().closure().callAsync(BROADCAST, @@ -4185,6 +4225,7 @@ public void awaitLastFut() { READ_COMMITTED, tCfg.getDefaultTxTimeout(), !ctx.skipStore(), + false, 0, null ); @@ -4286,6 +4327,7 @@ private IgniteInternalFuture asyncOp(final AsyncOp op) { READ_COMMITTED, txCfg.getDefaultTxTimeout(), !skipStore, + false, 0, null); @@ -4556,6 +4598,9 @@ public void onIgfsDataSizeChanged(long delta) { * @param readers Whether to clear readers. */ private boolean clearLocally0(K key, boolean readers) { + //TODO IGNITE-7952 + MvccUtils.verifyMvccOperationSupport(ctx, "Clear"); + ctx.checkSecurity(SecurityPermission.CACHE_REMOVE); if (keyCheck) @@ -4593,6 +4638,9 @@ private boolean clearLocally0(K key, boolean readers) { if (keyCheck) validateCacheKey(key); + //TODO IGNITE-7956 + MvccUtils.verifyMvccOperationSupport(ctx, "Evict"); + return evictx(key, ctx.versions().next(), CU.empty0()); } @@ -4606,6 +4654,9 @@ private boolean clearLocally0(K key, boolean readers) { if (keyCheck) validateCacheKey(keys); + //TODO IGNITE-7956 + MvccUtils.verifyMvccOperationSupport(ctx, "Evict"); + GridCacheVersion obsoleteVer = ctx.versions().next(); try { @@ -4890,7 +4941,8 @@ private void advance() { /*transformClo*/null, /*taskName*/null, /*expiryPlc*/null, - !deserializeBinary); + !deserializeBinary, + null); // TODO IGNITE-7371 if (val == null) return null; @@ -4950,6 +5002,7 @@ public void execute(boolean retry) { READ_COMMITTED, CU.transactionConfiguration(ctx, ctx.kernalContext().config()).getDefaultTxTimeout(), opCtx == null || !opCtx.skipStore(), + false, 0, null); 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 c9ee38cf898d6..cf4344d9b74b1 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 @@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityAssignment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.F; import org.jetbrains.annotations.Nullable; @@ -243,6 +244,10 @@ public AffinityAssignment assignment(AffinityTopologyVersion topVer) { return aff0.cachedAffinity(topVer); } + public MvccCoordinator mvccCoordinator(AffinityTopologyVersion topVer) { + return assignment(topVer).mvccCoordinator(); + } + /** * @param key Key to check. * @param topVer Topology version. 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 61b187883bb13..ac9de7ce09444 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 @@ -2113,6 +2113,13 @@ public boolean readNoEntry(@Nullable IgniteCacheExpiryPolicy expiryPlc, boolean return !config().isOnheapCacheEnabled() && !readers && expiryPlc == null; } + /** + * @return {@code True} if mvcc is enabled for cache. + */ + public boolean mvccEnabled() { + return grp.mvccEnabled(); + } + /** * @param part Partition. * @param topVer Topology version. 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 1445335cefda8..3a2af5dd9c187 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 @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache; import java.util.Collection; +import java.util.List; import java.util.UUID; import javax.cache.Cache; import javax.cache.expiry.ExpiryPolicy; @@ -27,7 +28,11 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -186,6 +191,11 @@ public interface GridCacheEntryEx { */ @Nullable public GridCacheEntryInfo info(); + /** + * @return Entry info for each MVCC version. + */ + @Nullable public List allVersionsInfo() throws IgniteCheckedException; + /** * Invalidates this entry. * @@ -264,7 +274,8 @@ public boolean evictInternal(GridCacheVersion obsoleteVer, @Nullable CacheEntryP Object transformClo, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) + boolean keepBinary, + @Nullable MvccSnapshot mvccVer) throws IgniteCheckedException, GridCacheEntryRemovedException; /** @@ -292,6 +303,7 @@ public EntryGetResult innerGetVersioned( String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean keepBinary, + @Nullable MvccSnapshot mvccVer, @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException; @@ -313,6 +325,7 @@ public EntryGetResult innerGetAndReserveForLoad(boolean updateMetrics, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean keepBinary, + @Nullable MvccSnapshot mvccVer, @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException; /** @@ -329,6 +342,64 @@ public EntryGetResult innerGetAndReserveForLoad(boolean updateMetrics, */ @Nullable public CacheObject innerReload() throws IgniteCheckedException, GridCacheEntryRemovedException; + /** + * @param tx Cache transaction. + * @param affNodeId Partitioned node iD. + * @param val Value to set. + * @param ttl0 TTL. + * @param topVer Topology version. + * @param updateCntr Update counter. + * @param mvccVer Mvcc version. + * @param op Cache operation. + * @param needHistory Whether to collect rows created or affected by the current tx. + * @param noCreate Entry should not be created when enabled, e.g. SQL INSERT. + * @return Tuple containing success flag and old value. If success is {@code false}, + * then value is {@code null}. + * @throws IgniteCheckedException If storing value failed. + * @throws GridCacheEntryRemovedException If entry has been removed. + */ + public GridCacheUpdateTxResult mvccSet( + @Nullable IgniteInternalTx tx, + UUID affNodeId, + CacheObject val, + long ttl0, + AffinityTopologyVersion topVer, + @Nullable Long updateCntr, + MvccSnapshot mvccVer, + GridCacheOperation op, + boolean needHistory, + boolean noCreate) throws IgniteCheckedException, GridCacheEntryRemovedException; + + /** + * @param tx Cache transaction. + * @param affNodeId Partitioned node iD. + * @param topVer Topology version. + * @param updateCntr Update counter. + * @param mvccVer Mvcc version. + * @param needHistory Whether to collect rows created or affected by the current tx. + * @return Tuple containing success flag and old value. If success is {@code false}, + * then value is {@code null}. + * @throws IgniteCheckedException If storing value failed. + * @throws GridCacheEntryRemovedException If entry has been removed. + */ + public GridCacheUpdateTxResult mvccRemove( + @Nullable IgniteInternalTx tx, + UUID affNodeId, + AffinityTopologyVersion topVer, + @Nullable Long updateCntr, + MvccSnapshot mvccVer, + boolean needHistory) throws IgniteCheckedException, GridCacheEntryRemovedException; + + /** + * @param tx Transaction adapter. + * @param mvccVer Mvcc version. + * @return Lock result. + * @throws GridCacheEntryRemovedException If entry has been removed. + * @throws IgniteCheckedException If locking failed + */ + GridCacheUpdateTxResult mvccLock(GridDhtTxLocalAdapter tx, + MvccSnapshot mvccVer) throws GridCacheEntryRemovedException, IgniteCheckedException; + /** * @param tx Cache transaction. * @param evtNodeId ID of node responsible for this change. @@ -377,7 +448,8 @@ public GridCacheUpdateTxResult innerSet( @Nullable UUID subjId, String taskName, @Nullable GridCacheVersion dhtVer, - @Nullable Long updateCntr + @Nullable Long updateCntr, + @Nullable MvccSnapshot mvccVer ) throws IgniteCheckedException, GridCacheEntryRemovedException; /** @@ -419,7 +491,8 @@ public GridCacheUpdateTxResult innerRemove( @Nullable UUID subjId, String taskName, @Nullable GridCacheVersion dhtVer, - @Nullable Long updateCntr + @Nullable Long updateCntr, + MvccSnapshot mvccVer ) throws IgniteCheckedException, GridCacheEntryRemovedException; /** @@ -663,8 +736,43 @@ public boolean tmLock(IgniteInternalTx tx, * @throws IgniteCheckedException In case of error. * @throws GridCacheEntryRemovedException If entry was removed. */ + default boolean initialValue(CacheObject val, + GridCacheVersion ver, + long ttl, + long expireTime, + boolean preload, + AffinityTopologyVersion topVer, + GridDrType drType, + boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException { + return initialValue(val, ver, null, null, TxState.NA, TxState.NA, + ttl, expireTime, preload, topVer, drType, fromStore); + } + + /** + * Sets new value if current version is 0 + * + * @param val New value. + * @param ver Version to use. + * @param mvccVer Mvcc version. + * @param newMvccVer New mvcc version. + * @param mvccTxState Tx state hint for mvcc version. + * @param newMvccTxState Tx state hint for new mvcc version. + * @param ttl Time to live. + * @param expireTime Expiration time. + * @param preload Flag indicating whether entry is being preloaded. + * @param topVer Topology version. + * @param drType DR type. + * @param fromStore {@code True} if value was loaded from store. + * @return {@code True} if initial value was set. + * @throws IgniteCheckedException In case of error. + * @throws GridCacheEntryRemovedException If entry was removed. + */ public boolean initialValue(CacheObject val, GridCacheVersion ver, + @Nullable MvccVersion mvccVer, + @Nullable MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState, long ttl, long expireTime, boolean preload, @@ -1044,6 +1152,27 @@ public void updateIndex(SchemaIndexCacheFilter filter, SchemaIndexCacheVisitorCl */ public boolean lockedByCurrentThread(); + /** + * + * @param tx Transaction. + * @param affNodeId Affinity node id. + * @param topVer Topology version. + * @param updateCntr Update counter. + * @param op Cache operation. + * @param mvccVer Mvcc version. @return Update result. + * @throws IgniteCheckedException, If failed. + * @throws GridCacheEntryRemovedException, If entry has been removed. + */ + public GridCacheUpdateTxResult mvccUpdateRowsWithPreloadInfo( + IgniteInternalTx tx, + UUID affNodeId, + AffinityTopologyVersion topVer, + Long updateCntr, + List entries, + GridCacheOperation op, + MvccSnapshot mvccVer) + throws IgniteCheckedException, GridCacheEntryRemovedException; + /** * Touch this entry in its context's eviction manager. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java index 7371153d4ae5d..5c8164c96a5d1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryInfo.java @@ -17,9 +17,11 @@ package org.apache.ignite.internal.processors.cache; -import java.nio.ByteBuffer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; 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; @@ -28,6 +30,8 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import java.nio.ByteBuffer; + /** * Entry information that gets passed over wire. */ 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 a1722db072b66..249fbb3a14ec7 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 @@ -34,6 +34,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.eviction.EvictableEntry; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.UnregisteredBinaryTypeException; import org.apache.ignite.internal.UnregisteredClassException; @@ -45,12 +46,17 @@ import org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult.UpdateOutcome; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.extras.GridCacheEntryExtras; import org.apache.ignite.internal.processors.cache.extras.GridCacheMvccEntryExtras; import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras; import org.apache.ignite.internal.processors.cache.extras.GridCacheTtlEntryExtras; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; @@ -59,6 +65,8 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.ResultType; import org.apache.ignite.internal.processors.cache.version.GridCacheLazyPlainVersionedEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; @@ -66,10 +74,14 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; import org.apache.ignite.internal.processors.dr.GridDrType; +import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFilter; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.IgniteTree; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridClosureException; +import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter; import org.apache.ignite.internal.util.lang.GridTuple; import org.apache.ignite.internal.util.lang.GridTuple3; @@ -82,6 +94,7 @@ 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.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; @@ -92,13 +105,19 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; +import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.NO_KEY; +import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.CONCURRENT_UPDATE; +import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.DUPLICATE_KEY; import static org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult.UpdateOutcome.INVOKE_NO_OP; import static org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult.UpdateOutcome.REMOVE_NO_VAL; +import static org.apache.ignite.internal.processors.dr.GridDrType.DR_BACKUP; import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; +import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRIMARY; /** * Adapter for cache entry. @@ -358,6 +377,77 @@ protected GridDhtLocalPartition localPartition() { return info; } + /** {@inheritDoc} */ + @Nullable @Override public List allVersionsInfo() throws IgniteCheckedException { + assert cctx.mvccEnabled(); + + lockEntry(); + + try { + if (obsolete()) + return Collections.emptyList(); + + GridCursor cur = + cctx.offheap().dataStore(localPartition()).mvccAllVersionsCursor(cctx, key, NO_KEY); + + List res = new ArrayList<>(); + + while (cur.next()) { + CacheDataRow row = cur.get(); + + GridCacheMvccEntryInfo info = new GridCacheMvccEntryInfo(); + + info.key(key); + info.value(row.value()); + info.cacheId(cctx.cacheId()); + info.version(row.version()); + info.setNew(false); + info.setDeleted(false); + + byte txState = row.mvccTxState() != TxState.NA ? row.mvccTxState() : + MvccUtils.state(cctx, row.mvccCoordinatorVersion(), row.mvccCounter(), + row.mvccOperationCounter()); + + if (txState == TxState.ABORTED) + continue; + + info.mvccVersion(row.mvccCoordinatorVersion(), row.mvccCounter(), row.mvccOperationCounter()); + info.mvccTxState(txState); + + byte newTxState = row.newMvccTxState() != TxState.NA ? row.newMvccTxState() : + MvccUtils.state(cctx, row.newMvccCoordinatorVersion(), row.newMvccCounter(), + row.newMvccOperationCounter()); + + if (newTxState != TxState.ABORTED) { + info.newMvccVersion(row.newMvccCoordinatorVersion(), + row.newMvccCounter(), + row.newMvccOperationCounter()); + + info.newMvccTxState(newTxState); + } + + long expireTime = row.expireTime(); + + long ttl; + + ttl = expireTime == CU.EXPIRE_TIME_ETERNAL ? CU.TTL_ETERNAL : expireTime - U.currentTimeMillis(); + + if (ttl < 0) + ttl = CU.TTL_MINIMUM; + + info.ttl(ttl); + info.expireTime(expireTime); + + res.add(info); + } + + return res; + } + finally { + unlockEntry(); + } + } + /** {@inheritDoc} */ @Override public final CacheObject unswap() throws IgniteCheckedException, GridCacheEntryRemovedException { return unswap(true); @@ -490,7 +580,8 @@ protected IgniteBiTuple valueBytes0() { Object transformClo, String taskName, @Nullable IgniteCacheExpiryPolicy expirePlc, - boolean keepBinary) + boolean keepBinary, + MvccSnapshot mvccVer) throws IgniteCheckedException, GridCacheEntryRemovedException { return (CacheObject)innerGet0( ver, @@ -505,6 +596,7 @@ protected IgniteBiTuple valueBytes0() { false, keepBinary, false, + mvccVer, null); } @@ -515,6 +607,7 @@ protected IgniteBiTuple valueBytes0() { String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean keepBinary, + MvccSnapshot mvccVer, @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException { return (EntryGetResult)innerGet0( /*ver*/null, @@ -529,6 +622,7 @@ protected IgniteBiTuple valueBytes0() { true, keepBinary, /*reserve*/true, + mvccVer, readerArgs); } @@ -543,6 +637,7 @@ protected IgniteBiTuple valueBytes0() { String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean keepBinary, + MvccSnapshot mvccVer, @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException { return (EntryGetResult)innerGet0( @@ -558,6 +653,7 @@ protected IgniteBiTuple valueBytes0() { true, keepBinary, false, + mvccVer, readerArgs); } @@ -576,6 +672,7 @@ private Object innerGet0( boolean retVer, boolean keepBinary, boolean reserveForLoad, + @Nullable MvccSnapshot mvccVer, @Nullable ReaderArguments readerArgs ) throws IgniteCheckedException, GridCacheEntryRemovedException { assert !(retVer && readThrough); @@ -599,41 +696,53 @@ private Object innerGet0( try { checkObsolete(); - boolean valid = valid(tx != null ? tx.topologyVersion() : cctx.affinity().affinityTopologyVersion()); - CacheObject val; - if (valid) { - val = this.val; - - if (val == null) { - if (isStartVersion()) { - unswap(null, false); + if (mvccVer != null) { + CacheDataRow row = cctx.offheap().mvccRead(cctx, key, mvccVer); - val = this.val; - } + if (row != null) { + val = row.value(); + resVer = row.version(); } + else + val = null; + } + else { + boolean valid = valid(tx != null ? tx.topologyVersion() : cctx.affinity().affinityTopologyVersion()); - if (val != null) { - long expireTime = expireTimeExtras(); + if (valid) { + val = this.val; - if (expireTime > 0 && (expireTime < U.currentTimeMillis())) { - if (onExpired((CacheObject)cctx.unwrapTemporary(val), null)) { - val = null; - evt = false; + if (val == null) { + if (isStartVersion()) { + unswap(null, false); - if (cctx.deferredDelete()) { - deferred = true; - ver0 = ver; + val = this.val; + } + } + + if (val != null) { + long expireTime = expireTimeExtras(); + + if (expireTime > 0 && (expireTime < U.currentTimeMillis())) { + if (onExpired((CacheObject)cctx.unwrapTemporary(val), null)) { + val = null; + evt = false; + + if (cctx.deferredDelete()) { + deferred = true; + ver0 = ver; + } + else + obsolete = true; } - else - obsolete = true; } } } + else + val = null; } - else - val = null; CacheObject ret = val; @@ -673,7 +782,7 @@ private Object innerGet0( if (ret != null && expiryPlc != null) updateTtl(expiryPlc); - if (retVer) { + if (retVer && resVer == null) { resVer = (isNear() && cctx.transactional()) ? ((GridNearCacheEntry)this).dhtVersion() : this.ver; if (resVer == null) @@ -759,7 +868,10 @@ else if (tx.dht()) { long expTime = CU.toExpireTime(ttl); // Update indexes before actual write to entry. - storeValue(ret, expTime, nextVer); + if (cctx.mvccEnabled()) + cctx.offheap().mvccInitialValue(this, ret, nextVer, expTime); + else + storeValue(ret, expTime, nextVer); update(ret, expTime, ttl, nextVer, true); @@ -870,13 +982,19 @@ private EntryGetResult entryGetResult(CacheObject val, GridCacheVersion ver, boo // Update indexes. if (ret != null) { - storeValue(ret, expTime, nextVer); + if (cctx.mvccEnabled()) + cctx.offheap().mvccInitialValue(this, ret, nextVer, expTime); + else + storeValue(ret, expTime, nextVer); if (cctx.deferredDelete() && !isInternal() && !detached() && deletedUnlocked()) deletedUnlocked(false); } else { - removeValue(); + if (cctx.mvccEnabled()) + cctx.offheap().mvccRemoveAll(this); + else + removeValue(); if (cctx.deferredDelete() && !isInternal() && !detached() && !deletedUnlocked()) deletedUnlocked(true); @@ -911,6 +1029,295 @@ protected void recordNodeId(UUID nodeId, AffinityTopologyVersion topVer) { // No-op. } + /** {@inheritDoc} */ + @Override public final GridCacheUpdateTxResult mvccSet( + IgniteInternalTx tx, + UUID affNodeId, + CacheObject val, + long ttl0, + AffinityTopologyVersion topVer, + @Nullable Long updateCntr, + MvccSnapshot mvccVer, + GridCacheOperation op, + boolean needHistory, + boolean noCreate) throws IgniteCheckedException, GridCacheEntryRemovedException { + assert tx != null; + + final boolean valid = valid(tx.topologyVersion()); + + final GridCacheVersion newVer; + + WALPointer logPtr = null; + + ensureFreeSpace(); + + lockEntry(); + + MvccUpdateResult res; + + try { + checkObsolete(); + + newVer = tx.writeVersion(); + + assert newVer != null : "Failed to get write version for tx: " + tx; + + // Determine new ttl and expire time. + long expireTime, ttl = ttl0; + + if (ttl == -1L) { + ttl = ttlExtras(); + expireTime = expireTimeExtras(); + } + else + expireTime = CU.toExpireTime(ttl); + + assert ttl >= 0 : ttl; + assert expireTime >= 0 : expireTime; + + // Detach value before index update. + val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); + + assert val != null; + + res = cctx.offheap().mvccUpdate( + this, val, newVer, expireTime, mvccVer, tx.local(), needHistory, noCreate); + + assert res != null; + + // VERSION_FOUND is possible only on primary node when inserting the same key, or on backup when + // updating the key which just has been rebalanced. + assert res.resultType() != ResultType.VERSION_FOUND || op == CREATE && tx.local() || !tx.local(); + + // PREV_NOT_NULL on CREATE is possible only on primary. + assert res.resultType() != ResultType.PREV_NOT_NULL || op != CREATE || tx.local(); + + if (res.resultType() == ResultType.VERSION_MISMATCH) + throw new IgniteSQLException("Mvcc version mismatch.", CONCURRENT_UPDATE); + else if (noCreate && res.resultType() == ResultType.PREV_NULL) + return new GridCacheUpdateTxResult(false); + else if (res.resultType() == ResultType.LOCKED) { + unlockEntry(); + + MvccVersion lockVer = res.resultVersion(); + + GridFutureAdapter resFut = new GridFutureAdapter<>(); + + IgniteInternalFuture lockFut = cctx.kernalContext().coordinators().waitFor(cctx, lockVer); + + lockFut.listen(new MvccUpdateLockListener(tx, this, affNodeId, topVer, val, ttl0, updateCntr, mvccVer, + op, needHistory, noCreate, resFut)); + + return new GridCacheUpdateTxResult(false, resFut); + } + else if (op == CREATE && tx.local() && (res.resultType() == ResultType.PREV_NOT_NULL || + res.resultType() == ResultType.VERSION_FOUND)) + throw new IgniteSQLException("Duplicate key during INSERT [key=" + key + ']', DUPLICATE_KEY); + + if (cctx.deferredDelete() && deletedUnlocked() && !detached()) + deletedUnlocked(false); + + assert tx.local() && updateCntr == null || !tx.local() && updateCntr != null && updateCntr > 0; + + if (tx.local()) + updateCntr = nextMvccPartitionCounter(); + + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) + logPtr = cctx.shared().wal().log(new DataRecord(new DataEntry( + cctx.cacheId(), + key, + val, + res.resultType() == ResultType.PREV_NULL ? CREATE : UPDATE, + tx.nearXidVersion(), + newVer, + expireTime, + key.partition(), + updateCntr))); + + update(val, expireTime, ttl, newVer, true); + + mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, val, newVer, topVer, mvccVer); + + recordNodeId(affNodeId, topVer); + } + finally { + if (lockedByCurrentThread()) { + unlockEntry(); + + cctx.evicts().touch(this, AffinityTopologyVersion.NONE); + } + } + + onUpdateFinished(updateCntr); + + GridCacheUpdateTxResult updRes = valid ? new GridCacheUpdateTxResult(true, updateCntr, logPtr) : + new GridCacheUpdateTxResult(false, logPtr); + + updRes.mvccHistory(res.history()); + + return updRes; + } + + /** {@inheritDoc} */ + @Override public final GridCacheUpdateTxResult mvccRemove( + IgniteInternalTx tx, + UUID affNodeId, + AffinityTopologyVersion topVer, + @Nullable Long updateCntr, + MvccSnapshot mvccVer, + boolean needHistory) throws IgniteCheckedException, GridCacheEntryRemovedException { + assert tx != null; + assert mvccVer != null; + + final boolean valid = valid(tx.topologyVersion()); + + final GridCacheVersion newVer; + + WALPointer logPtr = null; + + lockEntry(); + + MvccUpdateResult res; + + try { + checkObsolete(); + + newVer = tx.writeVersion(); + + assert newVer != null : "Failed to get write version for tx: " + tx; + + res = cctx.offheap().mvccRemove(this, mvccVer, tx.local(), needHistory); + + assert res != null; + + if (res.resultType() == ResultType.VERSION_MISMATCH) + throw new IgniteSQLException("Mvcc version mismatch.", CONCURRENT_UPDATE); + else if (res.resultType() == ResultType.PREV_NULL) + return new GridCacheUpdateTxResult(false); + else if (res.resultType() == ResultType.LOCKED) { + unlockEntry(); + + MvccVersion lockVer = res.resultVersion(); + + GridFutureAdapter resFut = new GridFutureAdapter<>(); + + IgniteInternalFuture lockFut = cctx.kernalContext().coordinators().waitFor(cctx, lockVer); + + lockFut.listen(new MvccRemoveLockListener(tx, this, affNodeId, topVer, updateCntr, mvccVer, needHistory, + resFut)); + + return new GridCacheUpdateTxResult(false, resFut); + } + + if (cctx.deferredDelete() && deletedUnlocked() && !detached()) + deletedUnlocked(false); + + assert tx.local() && updateCntr == null || !tx.local() && updateCntr != null && updateCntr > 0; + + if (tx.local()) + updateCntr = nextMvccPartitionCounter(); + + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) + logPtr = logTxUpdate(tx, null, 0, updateCntr); + + update(null, 0, 0, newVer, true); + + mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, null, newVer, topVer, mvccVer); + + recordNodeId(affNodeId, topVer); + } + finally { + if (lockedByCurrentThread()) { + unlockEntry(); + + cctx.evicts().touch(this, AffinityTopologyVersion.NONE); + } + } + + onUpdateFinished(updateCntr); + + GridCacheUpdateTxResult updRes = valid ? new GridCacheUpdateTxResult(true, updateCntr, logPtr) : + new GridCacheUpdateTxResult(false, logPtr); + + updRes.mvccHistory(res.history()); + + return updRes; + } + + /** {@inheritDoc} */ + @Override public GridCacheUpdateTxResult mvccLock(GridDhtTxLocalAdapter tx, MvccSnapshot mvccVer) + throws GridCacheEntryRemovedException, IgniteCheckedException { + assert tx != null; + assert mvccVer != null; + + final boolean valid = valid(tx.topologyVersion()); + + final GridCacheVersion newVer; + + WALPointer logPtr = null; + + lockEntry(); + + try { + checkObsolete(); + + newVer = tx.writeVersion(); + + assert newVer != null : "Failed to get write version for tx: " + tx; + + assert tx.local(); + + MvccUpdateResult res = cctx.offheap().mvccLock(this, mvccVer); + + assert res != null; + + if (res.resultType() == ResultType.VERSION_MISMATCH) + throw new IgniteSQLException("Mvcc version mismatch.", CONCURRENT_UPDATE); + else if (res.resultType() == ResultType.LOCKED) { + unlockEntry(); + + MvccVersion lockVer = res.resultVersion(); + + GridFutureAdapter resFut = new GridFutureAdapter<>(); + + IgniteInternalFuture lockFut = cctx.kernalContext().coordinators().waitFor(cctx, lockVer); + + lockFut.listen(new MvccAcquireLockListener(tx, this, mvccVer, resFut)); + + return new GridCacheUpdateTxResult(false, resFut); + } + } + finally { + if (lockedByCurrentThread()) { + unlockEntry(); + + cctx.evicts().touch(this, AffinityTopologyVersion.NONE); + } + } + + onUpdateFinished(0L); + + return new GridCacheUpdateTxResult(valid, logPtr); + } + + /** + * Enlist for DR if needed. + * + * @param drType DR type. + * @param val Value. + * @param ver Version. + * @param topVer Topology version. + * @param mvccVer MVCC snapshot. + * @throws IgniteCheckedException In case of exception. + */ + private void mvccDrReplicate(GridDrType drType, CacheObject val, GridCacheVersion ver, + AffinityTopologyVersion topVer, + MvccSnapshot mvccVer) throws IgniteCheckedException { + + if (cctx.isDrEnabled() && drType != DR_NONE && !isInternal()) + cctx.dr().mvccReplicate(key, val, rawTtl(), rawExpireTime(), ver.conflictVersion(), drType, topVer, mvccVer); + } + /** {@inheritDoc} */ @Override public final GridCacheUpdateTxResult innerSet( @Nullable IgniteInternalTx tx, @@ -933,15 +1340,16 @@ protected void recordNodeId(UUID nodeId, AffinityTopologyVersion topVer) { @Nullable UUID subjId, String taskName, @Nullable GridCacheVersion dhtVer, - @Nullable Long updateCntr + @Nullable Long updateCntr, + @Nullable MvccSnapshot mvccVer ) throws IgniteCheckedException, GridCacheEntryRemovedException { CacheObject old; - boolean valid = valid(tx != null ? tx.topologyVersion() : topVer); + final boolean valid = valid(tx != null ? tx.topologyVersion() : topVer); // Lock should be held by now. if (!cctx.isAll(this, filter)) - return new GridCacheUpdateTxResult(false, null, null); + return new GridCacheUpdateTxResult(false); final GridCacheVersion newVer; @@ -955,6 +1363,8 @@ protected void recordNodeId(UUID nodeId, AffinityTopologyVersion topVer) { ensureFreeSpace(); + GridLongList mvccWaitTxs = null; + lockEntry(); try { @@ -965,7 +1375,7 @@ protected void recordNodeId(UUID nodeId, AffinityTopologyVersion topVer) { // It is possible that 'get' could load more recent value. if (!((GridNearCacheEntry)this).recordDhtVersion(dhtVer)) - return new GridCacheUpdateTxResult(false, null, logPtr); + return new GridCacheUpdateTxResult(false, logPtr); } assert tx == null || (!tx.local() && tx.onePhaseCommit()) || tx.ownsLock(this) : @@ -1001,7 +1411,7 @@ protected void recordNodeId(UUID nodeId, AffinityTopologyVersion topVer) { key0 = e.key(); if (interceptorVal == null) - return new GridCacheUpdateTxResult(false, (CacheObject)cctx.unwrapTemporary(old), logPtr); + return new GridCacheUpdateTxResult(false, logPtr); else if (interceptorVal != val0) val0 = cctx.unwrapTemporary(interceptorVal); @@ -1033,7 +1443,18 @@ else if (interceptorVal != val0) assert val != null; - storeValue(val, expireTime, newVer); + if (cctx.mvccEnabled()) { + assert mvccVer != null; + + mvccWaitTxs = cctx.offheap().mvccUpdateNative(tx.local(), + this, + val, + newVer, + expireTime, + mvccVer); + } + else + storeValue(val, expireTime, newVer); if (cctx.deferredDelete() && deletedUnlocked() && !isInternal() && !detached()) deletedUnlocked(false); @@ -1112,8 +1533,8 @@ else if (interceptorVal != val0) if (intercept) cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, val, val0, keepBinary, updateCntr0)); - return valid ? new GridCacheUpdateTxResult(true, retval ? old : null, updateCntr0, logPtr) : - new GridCacheUpdateTxResult(false, null, logPtr); + return valid ? new GridCacheUpdateTxResult(true, updateCntr0, logPtr, mvccWaitTxs) : + new GridCacheUpdateTxResult(false, logPtr); } /** @@ -1142,7 +1563,8 @@ protected Object keyValue(boolean cpy) { @Nullable UUID subjId, String taskName, @Nullable GridCacheVersion dhtVer, - @Nullable Long updateCntr + @Nullable Long updateCntr, + @Nullable MvccSnapshot mvccVer ) throws IgniteCheckedException, GridCacheEntryRemovedException { assert cctx.transactional(); @@ -1150,11 +1572,11 @@ protected Object keyValue(boolean cpy) { GridCacheVersion newVer; - boolean valid = valid(tx != null ? tx.topologyVersion() : topVer); + final boolean valid = valid(tx != null ? tx.topologyVersion() : topVer); // Lock should be held by now. if (!cctx.isAll(this, filter)) - return new GridCacheUpdateTxResult(false, null, null); + return new GridCacheUpdateTxResult(false); GridCacheVersion obsoleteVer = null; @@ -1172,6 +1594,8 @@ protected Object keyValue(boolean cpy) { boolean marked = false; + GridLongList mvccWaitTxs = null; + lockEntry(); try { @@ -1182,7 +1606,7 @@ protected Object keyValue(boolean cpy) { // It is possible that 'get' could load more recent value. if (!((GridNearCacheEntry)this).recordDhtVersion(dhtVer)) - return new GridCacheUpdateTxResult(false, null, logPtr); + return new GridCacheUpdateTxResult(false, logPtr); } assert tx == null || (!tx.local() && tx.onePhaseCommit()) || tx.ownsLock(this) : @@ -1210,11 +1634,17 @@ protected Object keyValue(boolean cpy) { if (cctx.cancelRemove(interceptRes)) { CacheObject ret = cctx.toCacheObject(cctx.unwrapTemporary(interceptRes.get2())); - return new GridCacheUpdateTxResult(false, ret, logPtr); + return new GridCacheUpdateTxResult(false, logPtr); } } - removeValue(); + if (cctx.mvccEnabled()) { + assert mvccVer != null; + + mvccWaitTxs = cctx.offheap().mvccRemoveNative(tx.local(), this, mvccVer); + } + else + removeValue(); update(null, 0, 0, newVer, true); @@ -1336,18 +1766,10 @@ else if (log.isDebugEnabled()) if (intercept) cctx.config().getInterceptor().onAfterRemove(entry0); - if (valid) { - CacheObject ret; - - if (interceptRes != null) - ret = cctx.toCacheObject(cctx.unwrapTemporary(interceptRes.get2())); - else - ret = old; - - return new GridCacheUpdateTxResult(true, ret, updateCntr0, logPtr); - } + if (valid) + return new GridCacheUpdateTxResult(true, updateCntr0, logPtr, mvccWaitTxs); else - return new GridCacheUpdateTxResult(false, null, logPtr); + return new GridCacheUpdateTxResult(false, logPtr); } /** @@ -2176,7 +2598,10 @@ protected void clearReader(UUID nodeId) throws GridCacheEntryRemovedException { ", val=" + val + ']'); } - removeValue(); + if (cctx.mvccEnabled()) + cctx.offheap().mvccRemoveAll(this); + else + removeValue(); } finally { unlockEntry(); @@ -2408,7 +2833,10 @@ protected final boolean markObsolete0(GridCacheVersion ver, boolean clear, GridC ver = newVer; flags &= ~IS_EVICT_DISABLED; - removeValue(); + if (cctx.mvccEnabled()) + cctx.offheap().mvccRemoveAll(this); + else + removeValue(); onInvalidate(); @@ -2673,7 +3101,10 @@ private boolean checkExpired() throws IgniteCheckedException { long delta = expireTime - U.currentTimeMillis(); if (delta <= 0) { - removeValue(); + if (cctx.mvccEnabled()) + cctx.offheap().mvccRemoveAll(this); + else + removeValue(); return true; } @@ -2738,6 +3169,10 @@ protected final boolean hasValueUnlocked() { @Override public boolean initialValue( CacheObject val, GridCacheVersion ver, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState, long ttl, long expireTime, boolean preload, @@ -2812,11 +3247,63 @@ else if (val == null) } } - storeValue(val, expTime, ver); + if (cctx.mvccEnabled()) { + if (preload && mvccVer != null) { + cctx.offheap().mvccInitialValueIfAbsent(this, + val, + ver, + expTime, + mvccVer, + newMvccVer, + mvccTxState, + newMvccTxState); + } + else + cctx.offheap().mvccInitialValue(this, val, ver, expTime, mvccVer, newMvccVer); + } + else + storeValue(val, expTime, ver); } } - else // Optimization to access storage only once. - update = storeValue(val, expTime, ver, p); + else { + if (cctx.mvccEnabled()) { + // cannot identify whether the entry is exist on the fly + unswap(false); + + if (update = p.apply(null)) { + // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. + long oldExpTime = expireTimeUnlocked(); + long delta = (oldExpTime == 0 ? 0 : oldExpTime - U.currentTimeMillis()); + + if (delta < 0) { + if (onExpired(this.val, null)) { + if (cctx.deferredDelete()) { + deferred = true; + oldVer = this.ver; + } + else if (val == null) + obsolete = true; + } + } + + if (preload && mvccVer != null) { + cctx.offheap().mvccInitialValueIfAbsent(this, + val, + ver, + expTime, + mvccVer, + newMvccVer, + mvccTxState, + newMvccTxState); + } + else + cctx.offheap().mvccInitialValue(this, val, ver, expTime, mvccVer, newMvccVer); + } + } + else + // Optimization to access storage only once. + update = storeValue(val, expTime, ver, p); + } if (update) { update(val, expTime, ttl, ver, true); @@ -2917,6 +3404,14 @@ protected long nextPartitionCounter(AffinityTopologyVersion topVer, boolean prim return 0; } + /** + * @return Next mvcc update counter. + */ + protected long nextMvccPartitionCounter() { + return 0; + } + + /** {@inheritDoc} */ @Override public GridCacheVersionedEntryEx versionedEntry(final boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException { @@ -3004,7 +3499,10 @@ protected long nextPartitionCounter(AffinityTopologyVersion topVer, boolean prim // Detach value before index update. val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); - if (val != null) { + if (val != null) { + if (cctx.mvccEnabled()) + cctx.offheap().mvccInitialValue(this, val, newVer, expTime); + else storeValue(val, expTime, newVer); if (deletedUnlocked()) @@ -3459,7 +3957,10 @@ private boolean onExpired(CacheObject expiredVal, GridCacheVersion obsoleteVer) cctx.shared().database().checkpointReadLock(); try { - removeValue(); + if (cctx.mvccEnabled()) + cctx.offheap().mvccRemoveAll(this); + else + removeValue(); } finally { cctx.shared().database().checkpointReadUnlock(); @@ -3634,8 +4135,7 @@ protected boolean storeValue(@Nullable CacheObject val, } /** - * Stores value in offheap. - * + * Stores value inoffheap.* * @param val Value. * @param expireTime Expire time. * @param ver New entry version. @@ -4375,6 +4875,450 @@ private int extrasSize() { } } + /** */ + private static class MvccRemoveLockListener implements IgniteInClosure { + /** */ + private static final long serialVersionUID = -1578749008606139541L; + + /** */ + private final IgniteInternalTx tx; + + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final UUID affNodeId; + + /** */ + private final MvccSnapshot mvccVer; + + /** */ + private final Long updateCntr; + + /** */ + private final boolean needHistory; + + /** */ + private final GridFutureAdapter resFut; + + /** */ + private GridCacheMapEntry entry; + + /** */ + MvccRemoveLockListener(IgniteInternalTx tx, + GridCacheMapEntry entry, + UUID affNodeId, + AffinityTopologyVersion topVer, + Long updateCntr, + MvccSnapshot mvccVer, + boolean needHistory, + GridFutureAdapter resFut) { + this.tx = tx; + this.entry = entry; + this.topVer = topVer; + this.affNodeId = affNodeId; + this.mvccVer = mvccVer; + this.updateCntr = updateCntr; + this.needHistory = needHistory; + this.resFut = resFut; + } + + /** {@inheritDoc} */ + @Override public void apply(IgniteInternalFuture lockFut) { + WALPointer logPtr = null; + long updateCntr0; + boolean valid; + + GridCacheContext cctx = entry.context(); + GridCacheVersion newVer = tx.writeVersion(); + + MvccUpdateResult res; + + try { + lockFut.get(); + + while (true) { + entry.lockEntry(); + + if (entry.obsoleteVersionExtras() == null) + break; + + entry.unlockEntry(); + + entry = (GridCacheMapEntry)cctx.cache().entryEx(entry.key()); + } + + valid = entry.valid(tx.topologyVersion()); + + cctx.shared().database().checkpointReadLock(); + + try { + res = cctx.offheap().mvccRemove(entry, mvccVer, tx.local(), needHistory); + } finally { + cctx.shared().database().checkpointReadUnlock(); + } + + assert res != null; + + if (res.resultType() == ResultType.VERSION_MISMATCH) { + resFut.onDone(new IgniteSQLException("Mvcc version mismatch.", CONCURRENT_UPDATE)); + + return; + } + else if (res.resultType() == ResultType.PREV_NULL) { + resFut.onDone(new GridCacheUpdateTxResult(false)); + + return; + } + else if (res.resultType() == ResultType.LOCKED) { + entry.unlockEntry(); + + IgniteInternalFuture lockFuture = cctx.kernalContext().coordinators().waitFor(cctx, res.resultVersion()); + + lockFuture.listen(this); + + return; + } + + if (cctx.deferredDelete() && entry.deletedUnlocked() && !entry.detached()) + entry.deletedUnlocked(false); + + assert tx.local() && updateCntr == null || !tx.local() && updateCntr != null && updateCntr > 0; + + updateCntr0 = tx.local() ? entry.nextMvccPartitionCounter() : updateCntr; + + if (updateCntr != null && updateCntr != 0) + updateCntr0 = updateCntr; + + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) + logPtr = cctx.shared().wal().log(new DataRecord(new DataEntry( + cctx.cacheId(), + entry.key(), + null, + DELETE, + tx.nearXidVersion(), + tx.writeVersion(), + 0, + entry.key().partition(), + updateCntr0))); + + entry.update(null, 0, 0, newVer, true); + + entry.mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, null, newVer, topVer, mvccVer); + + entry.recordNodeId(affNodeId, topVer); + } + catch (IgniteCheckedException e) { + resFut.onDone(e); + + return; + } + finally { + if (entry.lockedByCurrentThread()) { + entry.unlockEntry(); + + cctx.evicts().touch(entry, AffinityTopologyVersion.NONE); + } + } + + entry.onUpdateFinished(updateCntr0); + + GridCacheUpdateTxResult updRes = valid ? new GridCacheUpdateTxResult(true, updateCntr0, logPtr) + : new GridCacheUpdateTxResult(false, logPtr); + + updRes.mvccHistory(res.history()); + + resFut.onDone(updRes); + } + } + + /** */ + private static class MvccAcquireLockListener implements IgniteInClosure { + /** */ + private static final long serialVersionUID = -1578749008606139541L; + + /** */ + private final IgniteInternalTx tx; + + /** */ + private final MvccSnapshot mvccVer; + + /** */ + private final GridFutureAdapter resFut; + + /** */ + private GridCacheMapEntry entry; + + /** */ + MvccAcquireLockListener(IgniteInternalTx tx, + GridCacheMapEntry entry, + MvccSnapshot mvccVer, + GridFutureAdapter resFut) { + this.tx = tx; + this.entry = entry; + this.mvccVer = mvccVer; + this.resFut = resFut; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void apply(IgniteInternalFuture lockFut) { + WALPointer logPtr = null; + boolean valid; + + GridCacheContext cctx = entry.context(); + + try { + lockFut.get(); + + while (true) { + entry.lockEntry(); + + if (entry.obsoleteVersionExtras() == null) + break; + + entry.unlockEntry(); + + entry = (GridCacheMapEntry)cctx.cache().entryEx(entry.key()); + } + + valid = entry.valid(tx.topologyVersion()); + + cctx.shared().database().checkpointReadLock(); + + MvccUpdateResult res; + + try { + res = cctx.offheap().mvccLock(entry, mvccVer); + } + finally { + cctx.shared().database().checkpointReadUnlock(); + } + + assert res != null; + + if (res.resultType() == ResultType.VERSION_MISMATCH) { + resFut.onDone(new IgniteSQLException("Mvcc version mismatch.", CONCURRENT_UPDATE)); + + return; + } + else if (res.resultType() == ResultType.LOCKED) { + entry.unlockEntry(); + + cctx.kernalContext().coordinators().waitFor(cctx, res.resultVersion()).listen(this); + + return; + } + } + catch (IgniteCheckedException e) { + resFut.onDone(e); + + return; + } + finally { + if (entry.lockedByCurrentThread()) { + entry.unlockEntry(); + + cctx.evicts().touch(entry, AffinityTopologyVersion.NONE); + } + } + + entry.onUpdateFinished(0L); + + resFut.onDone(new GridCacheUpdateTxResult(valid, logPtr)); + } + } + + /** */ + private static class MvccUpdateLockListener implements IgniteInClosure { + /** */ + private static final long serialVersionUID = 8452738214760268397L; + + /** */ + private final IgniteInternalTx tx; + + /** */ + private final UUID affNodeId; + + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final CacheObject val; + + /** */ + private final long ttl; + + /** */ + private final Long updateCntr; + + /** */ + private final MvccSnapshot mvccVer; + + /** */ + private final GridFutureAdapter resFut; + + /** */ + private GridCacheMapEntry entry; + + /** */ + private GridCacheOperation op; + + /** */ + private final boolean needHistory; + + /** */ + private final boolean noCreate; + + /** */ + MvccUpdateLockListener(IgniteInternalTx tx, + GridCacheMapEntry entry, + UUID affNodeId, + AffinityTopologyVersion topVer, + CacheObject val, + long ttl, + Long updateCntr, + MvccSnapshot mvccVer, + GridCacheOperation op, + boolean needHistory, + boolean noCreate, + GridFutureAdapter resFut) { + this.tx = tx; + this.entry = entry; + this.affNodeId = affNodeId; + this.topVer = topVer; + this.val = val; + this.ttl = ttl; + this.updateCntr = updateCntr; + this.mvccVer = mvccVer; + this.op = op; + this.needHistory = needHistory; + this.noCreate = noCreate; + this.resFut = resFut; + } + + /** {@inheritDoc} */ + @Override public void apply(IgniteInternalFuture lockFut) { + WALPointer logPtr = null; + long updateCntr0; + boolean valid; + + GridCacheContext cctx = entry.context(); + GridCacheVersion newVer = tx.writeVersion(); + + MvccUpdateResult res; + + try { + lockFut.get(); + + entry.ensureFreeSpace(); + + while (true) { + entry.lockEntry(); + + if (entry.obsoleteVersionExtras() == null) + break; + + entry.unlockEntry(); + + entry = (GridCacheMapEntry)cctx.cache().entryEx(entry.key()); + } + + valid = entry.valid(tx.topologyVersion()); + + // Determine new ttl and expire time. + long expireTime, ttl = this.ttl; + + if (ttl == -1L) { + ttl = entry.ttlExtras(); + expireTime = entry.expireTimeExtras(); + } + else + expireTime = CU.toExpireTime(ttl); + + assert ttl >= 0 : ttl; + assert expireTime >= 0 : expireTime; + + cctx.shared().database().checkpointReadLock(); + + try { + res = cctx.offheap().mvccUpdate( + entry, val, newVer, expireTime, mvccVer, tx.local(), needHistory, noCreate); + } finally { + cctx.shared().database().checkpointReadUnlock(); + } + + assert res != null; + + if (res.resultType() == ResultType.VERSION_MISMATCH) { + resFut.onDone(new IgniteSQLException("Mvcc version mismatch.", CONCURRENT_UPDATE)); + + return; + } + else if (res.resultType() == ResultType.LOCKED) { + entry.unlockEntry(); + + cctx.kernalContext().coordinators().waitFor(cctx, res.resultVersion()).listen(this); + + return; + } + else if (op == CREATE && tx.local() && (res.resultType() == ResultType.PREV_NOT_NULL || + res.resultType() == ResultType.VERSION_FOUND)) { + resFut.onDone(new IgniteSQLException("Duplicate key during INSERT [key=" + entry.key() + ']', + DUPLICATE_KEY)); + + return; + } + + if (cctx.deferredDelete() && entry.deletedUnlocked() && !entry.detached()) + entry.deletedUnlocked(false); + + assert tx.local() && updateCntr == null || !tx.local() && updateCntr != null && updateCntr > 0; + + updateCntr0 = tx.local() ? entry.nextMvccPartitionCounter() : updateCntr; + + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) + logPtr = cctx.shared().wal().log(new DataRecord(new DataEntry( + cctx.cacheId(), + entry.key(), + val, + res.resultType() == ResultType.PREV_NULL ? CREATE : UPDATE, + tx.nearXidVersion(), + newVer, + expireTime, + entry.key().partition(), + updateCntr0))); + + entry.update(val, expireTime, ttl, newVer, true); + + entry.mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, val, newVer, topVer, mvccVer); + + entry.recordNodeId(affNodeId, topVer); + } + catch (IgniteCheckedException e) { + resFut.onDone(e); + + return; + } + finally { + if (entry.lockedByCurrentThread()) { + entry.unlockEntry(); + + cctx.evicts().touch(entry, AffinityTopologyVersion.NONE); + } + } + + entry.onUpdateFinished(updateCntr0); + + GridCacheUpdateTxResult updRes = valid ? new GridCacheUpdateTxResult(true, updateCntr0, logPtr) + : new GridCacheUpdateTxResult(false, logPtr); + + updRes.mvccHistory(res.history()); + + resFut.onDone(updRes); + } + } + /** * */ @@ -5459,4 +6403,83 @@ private IgniteBiTuple runEntryProcessor(CacheInvokeEntry entries, + GridCacheOperation op, + MvccSnapshot mvccVer) + throws IgniteCheckedException, GridCacheEntryRemovedException { + assert updateCntr != null && updateCntr > 0 && !tx.local(); + + WALPointer logPtr = null; + + ensureFreeSpace(); + + lockEntry(); + + try { + checkObsolete(); + + CacheObject val = null; + + for (int i = 0; i < entries.size(); i++) { + GridCacheMvccEntryInfo info = (GridCacheMvccEntryInfo)entries.get(i); + + if (val == null && op != DELETE && MvccUtils.compare(info.mvccVersion(), + mvccVer.coordinatorVersion(), + mvccVer.counter(), + mvccVer.operationCounter()) == 0) + val = info.value(); + + cctx.offheap().mvccUpdateRowWithPreloadInfo(this, + info.value(), + info.version(), + info.expireTime(), + info.mvccVersion(), + info.newMvccVersion(), + info.mvccTxState(), + info.newMvccTxState()); + } + + if (cctx.deferredDelete() && deletedUnlocked() && !detached()) + deletedUnlocked(false); + + long expireTime = CU.EXPIRE_TIME_ETERNAL; + long ttl = CU.TTL_ETERNAL; + + GridCacheVersion ver = tx.writeVersion(); + + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) + logPtr = cctx.shared().wal().log(new DataRecord(new DataEntry( + cctx.cacheId(), + key, + val, + op, + tx.nearXidVersion(), + ver, + CU.EXPIRE_TIME_ETERNAL, + key.partition(), + updateCntr))); + + update(val, expireTime, ttl, ver, true); + + mvccDrReplicate(DR_BACKUP, val, ver, topVer, mvccVer); + + recordNodeId(affNodeId, topVer); + } + finally { + if (lockedByCurrentThread()) { + unlockEntry(); + + cctx.evicts().touch(this, AffinityTopologyVersion.NONE); + } + } + + return new GridCacheUpdateTxResult(true, logPtr); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java new file mode 100644 index 0000000000000..8df78eda2ea58 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java @@ -0,0 +1,257 @@ +/* + * 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.nio.ByteBuffer; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; +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 static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_BIT_OFF; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_MASK; + +/** + * + */ +public class GridCacheMvccEntryInfo extends GridCacheEntryInfo implements MvccVersionAware, MvccUpdateVersionAware { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long mvccCrdVer; + + /** */ + private long mvccCntr; + + /** */ + private int mvccOpCntr; + + /** */ + private long newMvccCrdVer; + + /** */ + private long newMvccCntr; + + /** */ + private int newMvccOpCntr; + + /** {@inheritDoc} */ + @Override public long newMvccCoordinatorVersion() { + return newMvccCrdVer; + } + + /** {@inheritDoc} */ + @Override public long newMvccCounter() { + return newMvccCntr; + } + + /** {@inheritDoc} */ + @Override public int newMvccOperationCounter() { + return newMvccOpCntr & ~MVCC_HINTS_MASK; + } + + /** {@inheritDoc} */ + @Override public byte newMvccTxState() { + return (byte)(newMvccOpCntr >>> MVCC_HINTS_BIT_OFF); + } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return mvccCrdVer; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return mvccCntr; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return mvccOpCntr & ~MVCC_HINTS_MASK; + } + + /** {@inheritDoc} */ + @Override public byte mvccTxState() { + return (byte)(mvccOpCntr >>> MVCC_HINTS_BIT_OFF); + } + + /** + * @param mvccTxState Mvcc version Tx state hint. + */ + public void mvccTxState(byte mvccTxState) { + mvccOpCntr = (mvccOpCntr & ~MVCC_HINTS_MASK) | ((int)mvccTxState << MVCC_HINTS_BIT_OFF); + } + + /** + * @param newMvccTxState New mvcc version Tx state hint. + */ + public void newMvccTxState(byte newMvccTxState) { + newMvccOpCntr = (newMvccOpCntr & ~MVCC_HINTS_MASK) | ((int)newMvccTxState << MVCC_HINTS_BIT_OFF); + } + + /** {@inheritDoc} */ + @Override public void newMvccVersion(long crd, long cntr, int opCntr) { + newMvccCrdVer = crd; + newMvccCntr = cntr; + newMvccOpCntr = opCntr; + } + + /** {@inheritDoc} */ + @Override public void mvccVersion(long crd, long cntr, int opCntr) { + mvccCrdVer = crd; + mvccCntr = cntr; + mvccOpCntr = opCntr; + } + + /** {@inheritDoc} */ + public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 6: + if (!writer.writeLong("mvccCntr", mvccCntr)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeLong("mvccCrdVer", mvccCrdVer)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeInt("mvccOpCntr", mvccOpCntr)) + return false; + + writer.incrementState(); + + case 9: + if (!writer.writeLong("newMvccCntr", newMvccCntr)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeLong("newMvccCrdVer", newMvccCrdVer)) + return false; + + writer.incrementState(); + + case 11: + if (!writer.writeInt("newMvccOpCntr", newMvccOpCntr)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 6: + mvccCntr = reader.readLong("mvccCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + mvccCrdVer = reader.readLong("mvccCrdVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: + mvccOpCntr = reader.readInt("mvccOpCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 9: + newMvccCntr = reader.readLong("newMvccCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: + newMvccCrdVer = reader.readLong("newMvccCrdVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: + newMvccOpCntr = reader.readInt("newMvccOpCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridCacheMvccEntryInfo.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 12; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 143; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridCacheMvccEntryInfo.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOperation.java index 555f825199eb2..377f95f6e4bd4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOperation.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOperation.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.processors.cache; -import org.jetbrains.annotations.*; +import org.jetbrains.annotations.Nullable; /** * Cache value operations. 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 9d88152e28ce3..ca2d1c8a888aa 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 @@ -894,7 +894,7 @@ public void lastFinishedFuture(GridDhtTopologyFuture fut) { * @param ver Topology version. * @return Future or {@code null} is future is already completed. */ - @Nullable public IgniteInternalFuture affinityReadyFuture(AffinityTopologyVersion ver) { + @Nullable public IgniteInternalFuture affinityReadyFuture(AffinityTopologyVersion ver) { GridDhtPartitionsExchangeFuture lastInitializedFut0 = lastInitializedFut; if (lastInitializedFut0 != null && lastInitializedFut0.initialVersion().compareTo(ver) == 0) { @@ -1873,9 +1873,15 @@ private void dumpPendingObjects(@Nullable AffinityTopologyVersion exchTopVer, IgniteTxManager tm = cctx.tm(); if (tm != null) { - U.warn(diagnosticLog, "Pending transactions:"); + boolean first = true; for (IgniteInternalTx tx : tm.activeTransactions()) { + if (first) { + U.warn(diagnosticLog, "Pending transactions:"); + + first = false; + } + if (exchTopVer != null) { U.warn(diagnosticLog, ">>> [txVer=" + tx.topologyVersionSnapshot() + ", exchWait=" + tm.needWaitTransaction(tx, exchTopVer) + @@ -1889,31 +1895,66 @@ private void dumpPendingObjects(@Nullable AffinityTopologyVersion exchTopVer, GridCacheMvccManager mvcc = cctx.mvcc(); if (mvcc != null) { - U.warn(diagnosticLog, "Pending explicit locks:"); + boolean first = true; + + for (GridCacheExplicitLockSpan lockSpan : mvcc.activeExplicitLocks()) { + if (first) { + U.warn(diagnosticLog, "Pending explicit locks:"); + + first = false; + } - for (GridCacheExplicitLockSpan lockSpan : mvcc.activeExplicitLocks()) U.warn(diagnosticLog, ">>> " + lockSpan); + } - U.warn(diagnosticLog, "Pending cache futures:"); + first = true; + + for (GridCacheFuture fut : mvcc.activeFutures()) { + if (first) { + U.warn(diagnosticLog, "Pending cache futures:"); + + first = false; + } - for (GridCacheFuture fut : mvcc.activeFutures()) dumpDiagnosticInfo(fut, diagCtx); + } - U.warn(diagnosticLog, "Pending atomic cache futures:"); + first = true; + + for (GridCacheFuture fut : mvcc.atomicFutures()) { + if (first) { + U.warn(diagnosticLog, "Pending atomic cache futures:"); + + first = false; + } - for (GridCacheFuture fut : mvcc.atomicFutures()) dumpDiagnosticInfo(fut, diagCtx); + } + + first = true; - U.warn(diagnosticLog, "Pending data streamer futures:"); + for (IgniteInternalFuture fut : mvcc.dataStreamerFutures()) { + if (first) { + U.warn(diagnosticLog, "Pending data streamer futures:"); + + first = false; + } - for (IgniteInternalFuture fut : mvcc.dataStreamerFutures()) dumpDiagnosticInfo(fut, diagCtx); + } if (tm != null) { - U.warn(diagnosticLog, "Pending transaction deadlock detection futures:"); + first = true; + + for (IgniteInternalFuture fut : tm.deadlockDetectionFutures()) { + if (first) { + U.warn(diagnosticLog, "Pending transaction deadlock detection futures:"); + + first = false; + } - for (IgniteInternalFuture fut : tm.deadlockDetectionFutures()) dumpDiagnosticInfo(fut, diagCtx); + } } } @@ -1935,6 +1976,8 @@ private void dumpPendingObjects(@Nullable AffinityTopologyVersion exchTopVer, affDumpCnt++; } } + + cctx.kernalContext().coordinators().dumpDebugInfo(diagnosticLog, diagCtx); } /** @@ -2070,12 +2113,21 @@ public boolean mergeExchangesOnCoordinator(GridDhtPartitionsExchangeFuture curFu ClusterNode node = evt.eventNode(); + if ((evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT) && + node.equals(cctx.coordinators().currentCoordinator())) { + if (log.isInfoEnabled()) + log.info("Stop merge, need exchange for mvcc coordinator failure: " + node); + + break; + } + if (!curFut.context().supportsMergeExchanges(node)) { if (log.isInfoEnabled()) log.info("Stop merge, node does not support merge: " + node); break; } + if (evt.type() == EVT_NODE_JOINED && cctx.cache().hasCachesReceivedFromJoin(node)) { if (log.isInfoEnabled()) log.info("Stop merge, received caches from node: " + node); 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 eb64d9c1d6628..e0a2420107bc6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -35,6 +35,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.stream.Collectors; +import javax.cache.expiry.EternalExpiryPolicy; import javax.management.MBeanServer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -97,7 +98,6 @@ import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; -import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; @@ -165,6 +165,7 @@ 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; +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; @@ -542,6 +543,24 @@ else if (cc.getRebalanceMode() == SYNC) { "other cache types [cacheName=" + cc.getName() + ", groupName=" + cc.getGroupName() + ", cacheType=" + cacheType + "]"); + if (cc.getAtomicityMode() == TRANSACTIONAL && c.isMvccEnabled()) { + if (cc.getCacheStoreFactory() != null) { + throw new IgniteCheckedException("Transactional cache may not have a third party cache store when " + + "MVCC is enabled."); + } + + if (cc.getExpiryPolicyFactory() != null && !(cc.getExpiryPolicyFactory().create() instanceof + EternalExpiryPolicy)) { + throw new IgniteCheckedException("Transactional cache may not have expiry policy when " + + "MVCC is enabled."); + } + + if (cc.getInterceptor() != null) { + throw new IgniteCheckedException("Transactional cache may not have an interceptor when " + + "MVCC is enabled."); + } + } + // Make sure we do not use sql schema for system views. if (ctx.query().moduleEnabled()) { String schema = QueryUtils.normalizeSchemaName(cc.getName(), cc.getSqlSchema()); @@ -1224,7 +1243,8 @@ private void startCache(GridCacheAdapter cache, QuerySchema schema) throws ", memoryPolicyName=" + memPlcName + ", mode=" + cfg.getCacheMode() + ", atomicity=" + cfg.getAtomicityMode() + - ", backups=" + cfg.getBackups() + ']'); + ", backups=" + cfg.getBackups() + + ", mvcc=" + cacheCtx.mvccEnabled() +']'); } } @@ -3408,7 +3428,7 @@ private Collection initiateCacheChanges( CacheConfiguration ccfg = req.startCacheConfiguration(); try { - cachesInfo.validateStartCacheConfiguration(ccfg); + cachesInfo.validateStartCacheConfiguration(ccfg, req.cacheType()); } catch (IgniteCheckedException e) { fut.onDone(e); @@ -4464,7 +4484,7 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( if (ccfg != null) { cloneCheckSerializable(ccfg); - if (desc != null || MetaStorage.METASTORAGE_CACHE_NAME.equals(cacheName)) { + if (desc != null) { if (failIfExists) { throw new CacheExistsException("Failed to start cache " + "(a cache with the same name is already started): " + cacheName); 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 4b98060d9e60a..bfe0001aafc65 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 @@ -37,7 +37,6 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.managers.communication.GridIoManager; import org.apache.ignite.internal.managers.deployment.GridDeploymentManager; -import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; @@ -48,6 +47,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionsEvictManager; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; @@ -793,6 +793,13 @@ public GridTimeoutProcessor time() { return kernalCtx.timeout(); } + /** + * @return Cache mvcc coordinator manager. + */ + public MvccProcessor coordinators() { + return kernalCtx.coordinators(); + } + /** * @return Partition evict manager. */ @@ -855,7 +862,7 @@ public IgniteLogger logger(String category) { /** * Captures all ongoing operations that we need to wait before we can proceed to the next topology version. * This method must be called only after - * {@link GridDhtPartitionTopology#updateTopologyVersion(GridDhtTopologyFuture, DiscoCache, long, boolean)} + * {@link GridDhtPartitionTopology#updateTopologyVersion} * method is called so that all new updates will wait to switch to the new version. * This method will capture: *
    diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java index 92af83b11fa91..b646cf9f1ed1a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java @@ -17,8 +17,12 @@ package org.apache.ignite.internal.processors.cache; +import java.util.List; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.pagemem.wal.WALPointer; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -29,26 +33,62 @@ public class GridCacheUpdateTxResult { /** Success flag.*/ private final boolean success; - /** Old value. */ - @GridToStringInclude - private final CacheObject oldVal; - - /** Partition idx. */ + /** Partition update counter. */ private long updateCntr; + /** */ + private GridLongList mvccWaitTxs; + + /** */ + private GridFutureAdapter fut; + /** */ private WALPointer logPtr; + /** */ + private List mvccHistory; + + /** + * Constructor. + * + * @param success Success flag. + */ + GridCacheUpdateTxResult(boolean success) { + this.success = success; + } + + /** + * Constructor. + * + * @param success Success flag. + * @param logPtr Logger WAL pointer for the update. + */ + GridCacheUpdateTxResult(boolean success, WALPointer logPtr) { + this.success = success; + this.logPtr = logPtr; + } + + /** + * Constructor. + * + * @param success Success flag. + * @param fut Update future. + */ + GridCacheUpdateTxResult(boolean success, GridFutureAdapter fut) { + this.success = success; + this.fut = fut; + } + /** * Constructor. * * @param success Success flag. - * @param oldVal Old value (if any), + * @param updateCntr Update counter. * @param logPtr Logger WAL pointer for the update. */ - GridCacheUpdateTxResult(boolean success, @Nullable CacheObject oldVal, WALPointer logPtr) { + GridCacheUpdateTxResult(boolean success, long updateCntr, WALPointer logPtr) { this.success = success; - this.oldVal = oldVal; + this.updateCntr = updateCntr; this.logPtr = logPtr; } @@ -56,20 +96,21 @@ public class GridCacheUpdateTxResult { * Constructor. * * @param success Success flag. - * @param oldVal Old value (if any). + * @param updateCntr Update counter. * @param logPtr Logger WAL pointer for the update. + * @param mvccWaitTxs List of transactions to wait for completion. */ - GridCacheUpdateTxResult(boolean success, @Nullable CacheObject oldVal, long updateCntr, WALPointer logPtr) { + GridCacheUpdateTxResult(boolean success, long updateCntr, WALPointer logPtr, GridLongList mvccWaitTxs) { this.success = success; - this.oldVal = oldVal; this.updateCntr = updateCntr; this.logPtr = logPtr; + this.mvccWaitTxs = mvccWaitTxs; } /** - * @return Partition idx. + * @return Partition update counter. */ - public long updatePartitionCounter() { + public long updateCounter() { return updateCntr; } @@ -88,10 +129,33 @@ public WALPointer loggedPointer() { } /** - * @return Old value. + * @return Update future. + */ + @Nullable public IgniteInternalFuture updateFuture() { + return fut; + } + + /** + * @return List of transactions to wait for completion. + */ + @Nullable public GridLongList mvccWaitTransactions() { + return mvccWaitTxs; + } + + /** + * + * @return Mvcc history rows. + */ + @Nullable public List mvccHistory() { + return mvccHistory; + } + + /** + * + * @param mvccHistory Mvcc history rows. */ - @Nullable public CacheObject oldValue() { - return oldVal; + public void mvccHistory(List mvccHistory) { + this.mvccHistory = mvccHistory; } /** {@inheritDoc} */ 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 213fb2e33ccca..9ef470cbf2461 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 @@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.LongAdder; import javax.cache.Cache; import javax.cache.CacheException; import javax.cache.configuration.Factory; @@ -67,10 +68,13 @@ import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; 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.near.GridNearCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog; +import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -99,7 +103,6 @@ import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.plugin.CachePluginConfiguration; import org.apache.ignite.plugin.security.SecurityException; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -173,6 +176,14 @@ public static boolean cheatCache(int id) { /** System cache name. */ public static final String UTILITY_CACHE_NAME = "ignite-sys-cache"; + /** Reserved cache names */ + public static final String[] RESERVED_NAMES = new String[] { + SYS_CACHE_HADOOP_MR, + UTILITY_CACHE_NAME, + MetaStorage.METASTORAGE_CACHE_NAME, + TxLog.TX_LOG_CACHE_NAME, + }; + /** */ public static final String CONTINUOUS_QRY_LOG_CATEGORY = "org.apache.ignite.continuous.query"; @@ -319,6 +330,18 @@ public static boolean cheatCache(int id) { } }; + /** Query mapped filter. */ + public static final IgnitePredicate FILTER_QUERY_MAPPING = new P1() { + + @Override public boolean apply(GridDistributedTxMapping m) { + return m.queryUpdate(); + } + + @Override public String toString() { + return "FILTER_QUERY_MAPPING"; + } + }; + /** Transaction entry to key. */ private static final IgniteClosure tx2key = new C1() { @Override public Object apply(IgniteTxEntry e) { @@ -588,6 +611,30 @@ public static IgniteReducer boolReducer() { }; } + /** + * @return Long reducer. + */ + public static IgniteReducer longReducer() { + return new IgniteReducer() { + private final LongAdder res = new LongAdder(); + + @Override public boolean collect(Long l) { + if(l != null) + res.add(l); + + return true; + } + + @Override public Long reduce() { + return res.sum(); + } + + @Override public String toString() { + return "Long reducer: " + res; + } + }; + } + /** * Gets reducer that aggregates maps into one. * @@ -1542,6 +1589,17 @@ public static void validateCacheName(String name) throws IllegalArgumentExceptio A.ensure(name != null && !name.isEmpty(), "Cache name must not be null or empty."); } + /** + * @param name Cache name. + * @throws IllegalArgumentException In case the name is not valid. + */ + public static void validateNewCacheName(String name) throws IllegalArgumentException { + validateCacheName(name); + + A.ensure(!isReservedCacheName(name), "Cache name cannot be \"" + name + + "\" because it is reserved for internal purposes."); + } + /** * @param cacheNames Cache names to validate. * @throws IllegalArgumentException In case the name is not valid. @@ -1558,7 +1616,20 @@ public static void validateCacheNames(Collection cacheNames) throws Ille public static void validateConfigurationCacheNames(Collection ccfgs) throws IllegalArgumentException { for (CacheConfiguration ccfg : ccfgs) - validateCacheName(ccfg.getName()); + validateNewCacheName(ccfg.getName()); + } + + /** + * @param name Cache name. + * @return {@code True} if it is a reserved cache name. + */ + public static boolean isReservedCacheName(String name) { + for (String reserved : RESERVED_NAMES) { + if (reserved.equals(name)) + return true; + } + + return false; } /** @@ -1829,6 +1900,44 @@ public static boolean isPersistenceEnabled(DataStorageConfiguration cfg) { return false; } + /** + * @param sctx Shared context. + * @param cacheIds Cache ids. + * @return First partitioned cache or {@code null} in case no partitioned cache ids are in list. + */ + public static GridCacheContext firstPartitioned(GridCacheSharedContext sctx, int[] cacheIds) { + for (int i = 0; i < cacheIds.length; i++) { + GridCacheContext cctx = sctx.cacheContext(cacheIds[i]); + + if (cctx == null) + throw new CacheException("Failed to find cache."); + + if (!cctx.isLocal() && !cctx.isReplicated()) + return cctx; + } + + return null; + } + + /** + * @param sctx Shared context. + * @param cacheIds Cache ids. + * @return First partitioned cache or {@code null} in case no partitioned cache ids are in list. + */ + public static GridCacheContext firstPartitioned(GridCacheSharedContext sctx, Iterable cacheIds) { + for (Integer i : cacheIds) { + GridCacheContext cctx = sctx.cacheContext(i); + + if (cctx == null) + throw new CacheException("Failed to find cache."); + + if (!cctx.isLocal() && !cctx.isReplicated()) + return cctx; + } + + return null; + } + /** * @param cacheName Name of cache or cache template. * @return {@code true} if cache name ends with asterisk (*), and therefire is a template name. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java index fa25412a81b17..a0213940de37e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java @@ -17,25 +17,33 @@ package org.apache.ignite.internal.processors.cache; +import java.util.List; import java.util.Map; import javax.cache.Cache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; import org.apache.ignite.internal.processors.cache.persistence.RootPage; import org.apache.ignite.internal.processors.cache.persistence.RowStore; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridAtomicLong; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.IgniteTree; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; +import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; /** @@ -168,6 +176,202 @@ public boolean expire(GridCacheContext cctx, IgniteInClosure2X> mvccAllVersions(GridCacheContext cctx, KeyCacheObject key) + throws IgniteCheckedException; + + /** + * Returns iterator over the all row versions for the given key. + * + * @param cctx Cache context. + * @param key Key. + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + * @return Iterator over all versions. + * @throws IgniteCheckedException If failed. + */ + GridCursor mvccAllVersionsCursor(GridCacheContext cctx, KeyCacheObject key, Object x) + throws IgniteCheckedException; + + /** + * @param entry Entry. + * @param val Value. + * @param ver Version. + * @param expireTime Expire time. + * @return {@code True} if value was inserted. + * @throws IgniteCheckedException If failed. + */ + default boolean mvccInitialValue( + GridCacheMapEntry entry, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime + ) throws IgniteCheckedException { + return mvccInitialValue(entry, val, ver, expireTime, null, null); + } + + /** + * @param entry Entry. + * @param val Value. + * @param ver Version. + * @param expireTime Expire time. + * @param mvccVer MVCC version. + * @param newMvccVer New MVCC version. + * @return {@code True} if value was inserted. + * @throws IgniteCheckedException If failed. + */ + public boolean mvccInitialValue( + GridCacheMapEntry entry, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer + ) throws IgniteCheckedException; + + /** + * @param entry Entry. + * @param val Value. + * @param ver Version. + * @param expireTime Expire time. + * @param mvccVer MVCC version. + * @param newMvccVer New MVCC version. + * @param txState Tx state hint for the mvcc version. + * @param newTxState Tx state hint for the new mvcc version. + * @return {@code True} if value was inserted. + * @throws IgniteCheckedException If failed. + */ + public boolean mvccInitialValueIfAbsent( + GridCacheMapEntry entry, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte txState, + byte newTxState + ) throws IgniteCheckedException; + + /** + * @param entry Entry. + * @param val Value. + * @param ver Cache version. + * @param expireTime Expire time. + * @param mvccSnapshot MVCC snapshot. + * @param primary {@code True} if on primary node. + * @param needHistory Flag to collect history. + * @param noCreate Flag indicating that row should not be created if absent. + * @return Update result. + * @throws IgniteCheckedException If failed. + */ + @Nullable public MvccUpdateResult mvccUpdate( + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory, + boolean noCreate) throws IgniteCheckedException; + + /** + * @param entry Entry. + * @param mvccSnapshot MVCC snapshot. + * @param primary {@code True} if on primary node. + * @param needHistory Flag to collect history. + * @return Update result. + * @throws IgniteCheckedException If failed. + */ + @Nullable public MvccUpdateResult mvccRemove( + GridCacheMapEntry entry, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory) throws IgniteCheckedException; + + /** + * @param entry Entry. + * @param mvccSnapshot MVCC snapshot. + * @return Update result. + * @throws IgniteCheckedException If failed. + */ + @Nullable public MvccUpdateResult mvccLock( + GridCacheMapEntry entry, + MvccSnapshot mvccSnapshot + ) throws IgniteCheckedException; + + /** + * @param entry Entry. + * @param val Value. + * @param ver Version. + * @param mvccVer MVCC version. + * @param newMvccVer New MVCC version. + * @return {@code True} if value was inserted. + * @throws IgniteCheckedException If failed. + */ + public boolean mvccUpdateRowWithPreloadInfo( + GridCacheMapEntry entry, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState + ) throws IgniteCheckedException; + + /** + * @param primary {@code True} if on primary node. + * @param entry Entry. + * @param val Value. + * @param ver Cache version. + * @param expireTime Expire time. + * @param mvccSnapshot MVCC snapshot. + * @return Transactions to wait for before finishing current transaction. + * @throws IgniteCheckedException If failed. + */ + GridLongList mvccUpdateNative( + boolean primary, + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException; + + /** + * @param primary {@code True} if on primary node. + * @param entry Entry. + * @param mvccSnapshot MVCC snapshot. + * @return Transactions to wait for before finishing current transaction. + * @throws IgniteCheckedException If failed. + */ + GridLongList mvccRemoveNative( + boolean primary, + GridCacheMapEntry entry, + MvccSnapshot mvccSnapshot + ) throws IgniteCheckedException; + + /** + * @param entry Entry. + * @throws IgniteCheckedException If failed. + */ + public void mvccRemoveAll(GridCacheMapEntry entry) + throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -209,6 +413,7 @@ public void remove( public int onUndeploy(ClassLoader ldr); /** + * * @param cacheId Cache ID. * @param primary Primary entries flag. * @param backup Backup entries flag. @@ -219,7 +424,8 @@ public void remove( public GridIterator cacheIterator(int cacheId, boolean primary, boolean backup, - final AffinityTopologyVersion topVer) + AffinityTopologyVersion topVer, + @Nullable MvccSnapshot mvccSnapshot) throws IgniteCheckedException; /** @@ -228,7 +434,20 @@ public GridIterator cacheIterator(int cacheId, * @return Partition data iterator. * @throws IgniteCheckedException If failed. */ - public GridIterator cachePartitionIterator(int cacheId, final int part) throws IgniteCheckedException; + public default GridIterator cachePartitionIterator(int cacheId, final int part) + throws IgniteCheckedException { + return cachePartitionIterator(cacheId, part, null); + } + + /** + * @param cacheId Cache ID. + * @param part Partition. + * @param mvccSnapshot MVCC snapshot. + * @return Partition data iterator. + * @throws IgniteCheckedException If failed. + */ + public GridIterator cachePartitionIterator(int cacheId, final int part, + @Nullable MvccSnapshot mvccSnapshot) throws IgniteCheckedException; /** * @param part Partition number. @@ -251,6 +470,7 @@ public GridCloseableIterator reservedIterator(final int part, fina * @return Partition data iterator. * @throws IgniteCheckedException If failed. */ + // TODO: MVCC> public IgniteRebalanceIterator rebalanceIterator(IgniteDhtDemandedPartitionsMap parts, AffinityTopologyVersion topVer) throws IgniteCheckedException; @@ -263,6 +483,7 @@ public IgniteRebalanceIterator rebalanceIterator(IgniteDhtDemandedPartitionsMap * @return Entries iterator. * @throws IgniteCheckedException If failed. */ + // TODO: MVCC> public GridCloseableIterator> cacheEntriesIterator( GridCacheContext cctx, final boolean primary, @@ -276,6 +497,7 @@ public GridCloseableIterator> cacheEntriesIterator( * @return Iterator. * @throws IgniteCheckedException If failed. */ + // TODO: MVCC> public GridCloseableIterator cacheKeysIterator(int cacheId, final int part) throws IgniteCheckedException; @@ -287,6 +509,7 @@ public GridCloseableIterator cacheKeysIterator(int cacheId, fina * @return Entries count. * @throws IgniteCheckedException If failed. */ + // TODO: MVCC> public long cacheEntriesCount(int cacheId, boolean primary, boolean backup, AffinityTopologyVersion topVer) throws IgniteCheckedException; @@ -402,7 +625,7 @@ interface CacheDataStore { long updateCounter(); /** - * + * @param val Update counter. */ void updateCounter(long val); @@ -411,6 +634,16 @@ interface CacheDataStore { */ public long nextUpdateCounter(); + /** + * @return Next mvcc update counter. + */ + long nextMvccUpdateCounter(); + + /** + * @return Current mvcc update counter value. + */ + long mvccUpdateCounter(); + /** * @return Initial update counter. */ @@ -434,6 +667,24 @@ CacheDataRow createRow( long expireTime, @Nullable CacheDataRow oldRow) throws IgniteCheckedException; + /** + * @param cctx Cache context. + * @param cleanupRows Rows to cleanup. + * @throws IgniteCheckedException If failed. + * @return Cleaned rows count. + */ + public int cleanup(GridCacheContext cctx, @Nullable List cleanupRows) + throws IgniteCheckedException; + + /** + * + * @param cctx Cache context. + * @param row Row. + * @throws IgniteCheckedException + */ + public void updateTxState(GridCacheContext cctx, CacheSearchRow row) + throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -451,6 +702,163 @@ void update( long expireTime, @Nullable CacheDataRow oldRow) throws IgniteCheckedException; + /** + * @param cctx Cache context. + * @param key Key. + * @param val Value. + * @param ver Version. + * @param mvccVer MVCC version. + * @param newMvccVer New MVCC version. + * @return {@code True} if new value was inserted. + * @throws IgniteCheckedException If failed. + */ + boolean mvccInitialValue( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param key Key. + * @param val Value. + * @param ver Version. + * @param mvccVer MVCC version. + * @param newMvccVer New MVCC version. + * @param txState Tx state hint for the mvcc version. + * @param newTxState Tx state hint for the new mvcc version. + * @return {@code True} if new value was inserted. + * @throws IgniteCheckedException If failed. + */ + boolean mvccInitialValueIfAbsent( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte txState, + byte newTxState) throws IgniteCheckedException; + + /** + * + * @param cctx Grid cache context. + * @param key Key. + * @param val Value. + * @param ver Version. + * @param expireTime Expiration time. + * @param mvccVer Mvcc version. + * @param newMvccVer New mvcc version. + * @return {@code true} on success. + * @throws IgniteCheckedException, if failed. + */ + boolean mvccUpdateRowWithPreloadInfo( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param key Key. + * @param val Value. + * @param ver Version. + * @param expireTime Expire time. + * @param mvccSnapshot MVCC snapshot. + * @param primary {@code True} if update is executed on primary node. + * @param needHistory Flag to collect history. + * @param noCreate Flag indicating that row should not be created if absent. + * @return Update result. + * @throws IgniteCheckedException If failed. + */ + MvccUpdateResult mvccUpdate( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory, + boolean noCreate) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param key Key. + * @param mvccSnapshot MVCC snapshot. + * @param primary {@code True} if update is executed on primary node. + * @param needHistory Flag to collect history. + * @return List of transactions to wait for. + * @throws IgniteCheckedException If failed. + */ + MvccUpdateResult mvccRemove( + GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param key Key. + * @param mvccSnapshot MVCC snapshot. + * @return List of transactions to wait for. + * @throws IgniteCheckedException If failed. + */ + MvccUpdateResult mvccLock( + GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param primary {@code True} if update is executed on primary node. + * @param key Key. + * @param val Value. + * @param ver Version. + * @param expireTime Expire time. + * @param mvccSnapshot MVCC snapshot. + * @return Update result. + * @throws IgniteCheckedException If failed. + */ + @Nullable GridLongList mvccUpdateNative( + GridCacheContext cctx, + boolean primary, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param primary {@code True} if update is executed on primary node. + * @param key Key. + * @param mvccSnapshot MVCC snapshot. + * @return List of transactions to wait for. + * @throws IgniteCheckedException If failed. + */ + @Nullable GridLongList mvccRemoveNative(GridCacheContext cctx, + boolean primary, + KeyCacheObject key, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param key Key. + * @throws IgniteCheckedException If failed. + */ + void mvccRemoveAll(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -475,12 +883,58 @@ void update( */ public CacheDataRow find(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException; + /** + * Returns iterator over the all row versions for the given key. + * + * @param cctx Cache context. + * @param key Key. + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + * @return Iterator over all versions. + * @throws IgniteCheckedException If failed. + */ + GridCursor mvccAllVersionsCursor(GridCacheContext cctx, KeyCacheObject key, Object x) + throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param key Key. + * @return Data row. + * @throws IgniteCheckedException If failed. + */ + public CacheDataRow mvccFind(GridCacheContext cctx, KeyCacheObject key, MvccSnapshot snapshot) + throws IgniteCheckedException; + + /** + * For testing only. + * + * @param cctx Cache context. + * @param key Key. + * @return All stored versions for given key. + * @throws IgniteCheckedException If failed. + */ + List> mvccFindAllVersions(GridCacheContext cctx, KeyCacheObject key) + throws IgniteCheckedException; + /** * @return Data cursor. * @throws IgniteCheckedException If failed. */ public GridCursor cursor() throws IgniteCheckedException; + /** + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + * @return Data cursor. + * @throws IgniteCheckedException If failed. + */ + public GridCursor cursor(Object x) throws IgniteCheckedException; + + /** + * @param mvccSnapshot MVCC snapshot. + * @return Data cursor. + * @throws IgniteCheckedException If failed. + */ + public GridCursor cursor(MvccSnapshot mvccSnapshot) throws IgniteCheckedException; + /** * @param cacheId Cache ID. * @return Data cursor. @@ -488,6 +942,15 @@ void update( */ public GridCursor cursor(int cacheId) throws IgniteCheckedException; + /** + * @param cacheId Cache ID. + * @param mvccSnapshot Mvcc snapshot. + * @return Data cursor. + * @throws IgniteCheckedException If failed. + */ + public GridCursor cursor(int cacheId, MvccSnapshot mvccSnapshot) + throws IgniteCheckedException; + /** * @param cacheId Cache ID. * @param lower Lower bound. @@ -509,6 +972,18 @@ public GridCursor cursor(int cacheId, KeyCacheObject low public GridCursor cursor(int cacheId, KeyCacheObject lower, KeyCacheObject upper, Object x) throws IgniteCheckedException; + /** + * @param cacheId Cache ID. + * @param lower Lower bound. + * @param upper Upper bound. + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + * @param snapshot Mvcc snapshot. + * @return Data cursor. + * @throws IgniteCheckedException If failed. + */ + public GridCursor cursor(int cacheId, KeyCacheObject lower, + KeyCacheObject upper, Object x, MvccSnapshot snapshot) throws IgniteCheckedException; + /** * Destroys the tree associated with the store. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index a8ad32d5a2032..2fe097c3c5865 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -17,10 +17,12 @@ 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.NoSuchElementException; import java.util.Set; @@ -36,6 +38,9 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.pagemem.FullPageId; +import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccMarkUpdatedRecord; +import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccUpdateNewTxStateHintRecord; +import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccUpdateTxStateHintRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; @@ -43,28 +48,48 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteHistoricalIterator; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteRebalanceIteratorImpl; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; import org.apache.ignite.internal.processors.cache.persistence.RootPage; import org.apache.ignite.internal.processors.cache.persistence.RowStore; import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; +import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.cache.tree.CacheDataRowStore; import org.apache.ignite.internal.processors.cache.tree.CacheDataTree; import org.apache.ignite.internal.processors.cache.tree.DataRow; import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; import org.apache.ignite.internal.processors.cache.tree.PendingRow; +import org.apache.ignite.internal.processors.cache.tree.RowLinkIO; import org.apache.ignite.internal.processors.cache.tree.SearchRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateDataRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateDataRowNative; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.ResultType; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccFirstRowTreeClosure; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccMaxSearchRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccMinSearchRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccSnapshotSearchRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccTreeClosure; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.GridEmptyCloseableIterator; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.GridStripedLock; +import org.apache.ignite.internal.util.IgniteTree; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.lang.GridIterator; @@ -72,6 +97,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; 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.IgnitePredicate; @@ -80,6 +106,22 @@ import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX; import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.INITIAL_VERSION; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.compare; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.compareNewVersion; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.isVisible; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccVersionIsValid; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.state; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.unexpectedStateException; +import static org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.EMPTY_CURSOR; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_BIT_OFF; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_MASK; +import static org.apache.ignite.internal.util.IgniteTree.OperationType.NOOP; +import static org.apache.ignite.internal.util.IgniteTree.OperationType.PUT; /** * @@ -396,6 +438,170 @@ private Iterator cacheData(boolean primary, boolean backup, Affi dataStore(part).update(cctx, key, val, ver, expireTime, oldRow); } + /** {@inheritDoc} */ + @Override public boolean mvccInitialValue( + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer) throws IgniteCheckedException { + return dataStore(entry.localPartition()).mvccInitialValue( + entry.context(), + entry.key(), + val, + ver, + expireTime, + mvccVer, + newMvccVer); + } + + /** {@inheritDoc} */ + @Override public boolean mvccInitialValueIfAbsent( + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte txState, + byte newTxState) throws IgniteCheckedException { + return dataStore(entry.localPartition()).mvccInitialValueIfAbsent( + entry.context(), + entry.key(), + val, + ver, + expireTime, + mvccVer, + newMvccVer, + txState, + newTxState); + } + + /** {@inheritDoc} */ + @Override public boolean mvccUpdateRowWithPreloadInfo( + GridCacheMapEntry entry, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState + ) throws IgniteCheckedException { + assert entry.lockedByCurrentThread(); + + return dataStore(entry.localPartition()).mvccUpdateRowWithPreloadInfo( + entry.context(), + entry.key(), + val, + ver, + expireTime, + mvccVer, + newMvccVer, + mvccTxState, + newMvccTxState + ); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccUpdate( + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory, + boolean noCreate) throws IgniteCheckedException { + if (entry.detached() || entry.isNear()) + return null; + + assert entry.lockedByCurrentThread(); + + return dataStore(entry.localPartition()).mvccUpdate(entry.context(), + entry.key(), + val, + ver, + expireTime, + mvccSnapshot, + primary, + needHistory, + noCreate); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccRemove( + GridCacheMapEntry entry, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory) throws IgniteCheckedException { + if (entry.detached() || entry.isNear()) + return null; + + assert entry.lockedByCurrentThread(); + + return dataStore(entry.localPartition()).mvccRemove(entry.context(), + entry.key(), + mvccSnapshot, + primary, + needHistory); + } + + /** {@inheritDoc} */ + @Override public GridLongList mvccUpdateNative( + boolean primary, + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + if (entry.detached() || entry.isNear()) + return null; + + return dataStore(entry.localPartition()).mvccUpdateNative(entry.context(), + primary, + entry.key(), + val, + ver, + expireTime, + mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public GridLongList mvccRemoveNative( + boolean primary, + GridCacheMapEntry entry, + MvccSnapshot mvccSnapshot + ) throws IgniteCheckedException { + if (entry.detached() || entry.isNear()) + return null; + + return dataStore(entry.localPartition()).mvccRemoveNative(entry.context(), + primary, + entry.key(), + mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public void mvccRemoveAll(GridCacheMapEntry entry) throws IgniteCheckedException { + if (entry.detached() || entry.isNear()) + return; + + dataStore(entry.localPartition()).mvccRemoveAll(entry.context(), entry.key()); + } + + /** {@inheritDoc} */ + @Nullable @Override public MvccUpdateResult mvccLock(GridCacheMapEntry entry, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + if (entry.detached() || entry.isNear()) + return null; + + assert entry.lockedByCurrentThread(); + + return dataStore(entry.localPartition()).mvccLock(entry.context(), entry.key(), mvccSnapshot); + } + /** {@inheritDoc} */ @Override public void remove( GridCacheContext cctx, @@ -418,22 +624,62 @@ private Iterator cacheData(boolean primary, boolean backup, Affi } /** {@inheritDoc} */ - @Nullable @Override public CacheDataRow read(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { - CacheDataRow row; + @Nullable @Override public CacheDataRow read(GridCacheContext cctx, KeyCacheObject key) + throws IgniteCheckedException { + CacheDataStore dataStore = dataStore(cctx, key); - if (cctx.isLocal()) - row = locCacheDataStore.find(cctx, key); - else { - GridDhtLocalPartition part = cctx.topology().localPartition(cctx.affinity().partition(key), null, false); + CacheDataRow row = dataStore != null ? dataStore.find(cctx, key) : null; - row = part != null ? dataStore(part).find(cctx, key) : null; - } + assert row == null || row.value() != null : row; + + return row; + } + + /** {@inheritDoc} */ + @Nullable @Override public CacheDataRow mvccRead(GridCacheContext cctx, KeyCacheObject key, MvccSnapshot ver) + throws IgniteCheckedException { + assert ver != null; + + CacheDataStore dataStore = dataStore(cctx, key); + + CacheDataRow row = dataStore != null ? dataStore.mvccFind(cctx, key, ver) : null; assert row == null || row.value() != null : row; return row; } + /** {@inheritDoc} */ + @Override public List> mvccAllVersions(GridCacheContext cctx, KeyCacheObject key) + throws IgniteCheckedException { + CacheDataStore dataStore = dataStore(cctx, key); + + return dataStore != null ? dataStore.mvccFindAllVersions(cctx, key) : + Collections.emptyList(); + } + + /** {@inheritDoc} */ + @Override public GridCursor mvccAllVersionsCursor(GridCacheContext cctx, + KeyCacheObject key, Object x) throws IgniteCheckedException { + CacheDataStore dataStore = dataStore(cctx, key); + + return dataStore != null ? dataStore.mvccAllVersionsCursor(cctx, key, x) : EMPTY_CURSOR; + } + + /** + * @param cctx Cache context. + * @param key Key. + * @return Data store. + */ + @Nullable private CacheDataStore dataStore(GridCacheContext cctx, KeyCacheObject key) { + if (grp.isLocal()) + return locCacheDataStore; + + GridDhtLocalPartition part = grp.topology().localPartition(cctx.affinity().partition(key), null, false); + + return part != null ? dataStore(part) : null; + } + /** {@inheritDoc} */ @Override public boolean containsKey(GridCacheMapEntry entry) { try { @@ -470,7 +716,7 @@ private Iterator cacheData(boolean primary, boolean backup, Affi @Override public void clearCache(GridCacheContext cctx, boolean readers) { GridCacheVersion obsoleteVer = null; - try (GridCloseableIterator it = grp.isLocal() ? iterator(cctx.cacheId(), cacheDataStores().iterator()) : + try (GridCloseableIterator it = grp.isLocal() ? iterator(cctx.cacheId(), cacheDataStores().iterator(), null) : evictionSafeIterator(cctx.cacheId(), cacheDataStores().iterator())) { while (it.hasNext()) { cctx.shared().database().checkpointReadLock(); @@ -529,7 +775,7 @@ private Iterator cacheData(boolean primary, boolean backup, Affi final boolean backup, final AffinityTopologyVersion topVer, final boolean keepBinary) throws IgniteCheckedException { - final Iterator it = cacheIterator(cctx.cacheId(), primary, backup, topVer); + final Iterator it = cacheIterator(cctx.cacheId(), primary, backup, topVer, null); return new GridCloseableIteratorAdapter>() { /** */ @@ -570,7 +816,8 @@ private Iterator cacheData(boolean primary, boolean backup, Affi } /** {@inheritDoc} */ - @Override public GridCloseableIterator cacheKeysIterator(int cacheId, final int part) throws IgniteCheckedException { + @Override public GridCloseableIterator cacheKeysIterator(int cacheId, final int part) + throws IgniteCheckedException { CacheDataStore data = partitionData(part); if (data == null) @@ -611,19 +858,21 @@ private Iterator cacheData(boolean primary, boolean backup, Affi int cacheId, boolean primary, boolean backups, - final AffinityTopologyVersion topVer) + final AffinityTopologyVersion topVer, + @Nullable MvccSnapshot mvccSnapshot) throws IgniteCheckedException { - return iterator(cacheId, cacheData(primary, backups, topVer)); + return iterator(cacheId, cacheData(primary, backups, topVer), mvccSnapshot); } /** {@inheritDoc} */ - @Override public GridIterator cachePartitionIterator(int cacheId, int part) throws IgniteCheckedException { + @Override public GridIterator cachePartitionIterator(int cacheId, int part, + @Nullable MvccSnapshot mvccSnapshot) throws IgniteCheckedException { CacheDataStore data = partitionData(part); if (data == null) return new GridEmptyCloseableIterator<>(); - return iterator(cacheId, singletonIterator(data)); + return iterator(cacheId, singletonIterator(data), mvccSnapshot); } /** {@inheritDoc} */ @@ -633,15 +882,20 @@ private Iterator cacheData(boolean primary, boolean backup, Affi if (data == null) return new GridEmptyCloseableIterator<>(); - return iterator(CU.UNDEFINED_CACHE_ID, singletonIterator(data)); + return iterator(CU.UNDEFINED_CACHE_ID, singletonIterator(data), null); } /** + * * @param cacheId Cache ID. * @param dataIt Data store iterator. + * @param mvccSnapshot Mvcc snapshot. * @return Rows iterator */ - private GridCloseableIterator iterator(final int cacheId, final Iterator dataIt) { + private GridCloseableIterator iterator(final int cacheId, + final Iterator dataIt, + final MvccSnapshot mvccSnapshot) + { return new GridCloseableIteratorAdapter() { /** */ private GridCursor cur; @@ -670,7 +924,13 @@ private GridCloseableIterator iterator(final int cacheId, final It CacheDataStore ds = dataIt.next(); curPart = ds.partId(); - cur = cacheId == CU.UNDEFINED_CACHE_ID ? ds.cursor() : ds.cursor(cacheId); + + if (mvccSnapshot == null) + cur = cacheId == CU.UNDEFINED_CACHE_ID ? ds.cursor() : ds.cursor(cacheId); + else { + cur = cacheId == CU.UNDEFINED_CACHE_ID ? + ds.cursor(mvccSnapshot) : ds.cursor(cacheId, mvccSnapshot); + } } else break; @@ -1142,6 +1402,13 @@ protected class CacheDataStoreImpl implements CacheDataStore { /** Update counter. */ protected final AtomicLong cntr = new AtomicLong(); + /** + * Mvcc update counter. This counter is used for an mvcc-style entries updates where this counter is + * incremented on each entry write (which happens before commit), but main update counter is updated + * on commit phase only. + */ + protected final AtomicLong mvccUpdCntr = new AtomicLong(); + /** Partition size. */ private final AtomicLong storageSize = new AtomicLong(); @@ -1151,6 +1418,15 @@ protected class CacheDataStoreImpl implements CacheDataStore { /** Initial update counter. */ protected long initCntr; + /** Mvcc remove handler. */ + private final PageHandler mvccUpdateMarker = new MvccMarkUpdatedHandler(); + + /** Mvcc update tx state hint handler. */ + private final PageHandler mvccUpdateTxStateHint = new MvccUpdateTxStateHintHandler(); + + /** */ + private final PageHandler mvccApplyChanges = new MvccApplyChangesHandler(); + /** * @param partId Partition number. * @param name Name. @@ -1239,14 +1515,17 @@ void decrementSize(int cacheId) { return storageSize.get(); } + /** {@inheritDoc} */ + @Override public long nextUpdateCounter() { + return cntr.incrementAndGet(); + } + /** {@inheritDoc} */ @Override public long updateCounter() { return cntr.get(); } - /** - * @param val Update index value. - */ + /** {@inheritDoc} */ @Override public void updateCounter(long val) { while (true) { long val0 = cntr.get(); @@ -1259,6 +1538,16 @@ void decrementSize(int cacheId) { } } + /** {@inheritDoc} */ + @Override public long nextMvccUpdateCounter() { + return mvccUpdCntr.incrementAndGet(); + } + + /** {@inheritDoc} */ + @Override public long mvccUpdateCounter() { + return mvccUpdCntr.get(); + } + /** {@inheritDoc} */ @Override public String name() { return name; @@ -1273,7 +1562,7 @@ void decrementSize(int cacheId) { */ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow oldRow, DataRow dataRow) throws IgniteCheckedException { - if (oldRow == null || cctx.queries().enabled()) + if (oldRow == null || cctx.queries().enabled() || grp.mvccEnabled()) return false; if (oldRow.expireTime() != dataRow.expireTime()) @@ -1282,28 +1571,39 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol // Use grp.sharedGroup() flag since it is possible cacheId is not yet set here. boolean sizeWithCacheId = grp.sharedGroup(); - int oldLen = DataPageIO.getRowSize(oldRow, sizeWithCacheId); + int oldLen = oldRow.size(); if (oldLen > updateValSizeThreshold) return false; - int newLen = DataPageIO.getRowSize(dataRow, sizeWithCacheId); + int newLen = dataRow.size(); return oldLen == newLen; } /** {@inheritDoc} */ @Override public void invoke(GridCacheContext cctx, KeyCacheObject key, OffheapInvokeClosure c) + throws IgniteCheckedException { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + invoke0(cctx, new SearchRow(cacheId, key), c); + } + + /** + * @param cctx Cache context. + * @param row Search row. + * @param c Closure. + * @throws IgniteCheckedException If failed. + */ + private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClosure c) throws IgniteCheckedException { if (!busyLock.enterBusy()) throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); try { - int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - assert cctx.shared().database().checkpointLockIsHeldByThread(); - dataTree.invoke(new SearchRow(cacheId, key), CacheDataRowAdapter.RowData.NO_KEY, c); + dataTree.invoke(row, CacheDataRowAdapter.RowData.NO_KEY, c); switch (c.operationType()) { case PUT: { @@ -1319,7 +1619,7 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol case REMOVE: { CacheDataRow oldRow = c.oldRow(); - finishRemove(cctx, key, oldRow); + finishRemove(cctx, row.key(), oldRow); break; } @@ -1368,149 +1668,814 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol } /** {@inheritDoc} */ - @Override public void update( + @Override public boolean mvccInitialValue( GridCacheContext cctx, KeyCacheObject key, - CacheObject val, + @Nullable CacheObject val, GridCacheVersion ver, long expireTime, - @Nullable CacheDataRow oldRow - ) throws IgniteCheckedException { - assert oldRow == null || oldRow.link() != 0L : oldRow; + MvccVersion mvccVer, + MvccVersion newMvccVer) + throws IgniteCheckedException + { + assert mvccVer != null || newMvccVer == null : newMvccVer; if (!busyLock.enterBusy()) throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); try { - int cacheId = grp.storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - - assert oldRow == null || oldRow.cacheId() == cacheId : oldRow; - - if (key.partition() == -1) - key.partition(partId); - - DataRow dataRow = new DataRow(key, val, ver, partId, expireTime, cacheId); - CacheObjectContext coCtx = cctx.cacheObjectContext(); // Make sure value bytes initialized. key.valueBytes(coCtx); - val.valueBytes(coCtx); - CacheDataRow old; + // null is passed for loaded from store. + if (mvccVer == null) { + mvccVer = INITIAL_VERSION; - assert cctx.shared().database().checkpointLockIsHeldByThread(); + // Clean all versions of row + mvccRemoveAll(cctx, key); + } - if (canUpdateOldRow(cctx, oldRow, dataRow) && rowStore.updateRow(oldRow.link(), dataRow)) { - old = oldRow; + if (val != null) { + val.valueBytes(coCtx); - dataRow.link(oldRow.link()); - } - else { - rowStore.addRow(dataRow); + MvccDataRow updateRow = new MvccDataRow( + key, + val, + ver, + partId, + expireTime, + cctx.cacheId(), + mvccVer, + newMvccVer); - assert dataRow.link() != 0 : dataRow; + assert cctx.shared().database().checkpointLockIsHeldByThread(); - if (grp.sharedGroup() && dataRow.cacheId() == CU.UNDEFINED_CACHE_ID) - dataRow.cacheId(cctx.cacheId()); + if (!grp.storeCacheIdInDataPage() && updateRow.cacheId() != CU.UNDEFINED_CACHE_ID) { + updateRow.cacheId(CU.UNDEFINED_CACHE_ID); - if (oldRow != null) { - old = oldRow; + rowStore.addRow(updateRow); - dataTree.putx(dataRow); + updateRow.cacheId(cctx.cacheId()); } else - old = dataTree.put(dataRow); - } + rowStore.addRow(updateRow); - finishUpdate(cctx, dataRow, old); + dataTree.putx(updateRow); + + incrementSize(cctx.cacheId()); + + if (cctx.queries().enabled()) + cctx.queries().store(updateRow, null, true); + + return true; + } } finally { busyLock.leaveBusy(); } - } - /** - * @param cctx Cache context. - * @param newRow New row. - * @param oldRow Old row if available. - * @throws IgniteCheckedException If failed. - */ - private void finishUpdate(GridCacheContext cctx, CacheDataRow newRow, @Nullable CacheDataRow oldRow) - throws IgniteCheckedException { - if (oldRow == null) - incrementSize(cctx.cacheId()); + return false; + } - KeyCacheObject key = newRow.key(); + /** {@inheritDoc} */ + @Override public boolean mvccInitialValueIfAbsent( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte txState, + byte newTxState) + throws IgniteCheckedException + { + assert mvccVer != null; - long expireTime = newRow.expireTime(); + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); - GridCacheQueryManager qryMgr = cctx.queries(); + try { + CacheObjectContext coCtx = cctx.cacheObjectContext(); - int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + // Make sure value bytes initialized. + key.valueBytes(coCtx); - if (qryMgr.enabled()) - qryMgr.store(newRow, oldRow, true); + if (val != null) + val.valueBytes(coCtx); - if (oldRow != null) { - assert oldRow.link() != 0 : oldRow; + assert cctx.shared().database().checkpointLockIsHeldByThread(); - if (pendingTree() != null && oldRow.expireTime() != 0) - pendingTree().removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); + MvccPutIfAbsentClosure clo = new MvccPutIfAbsentClosure(key, + val, + ver, + partId, + expireTime, + cctx.cacheId(), + mvccVer, + newMvccVer, + txState, + newTxState); - if (newRow.link() != oldRow.link()) - rowStore.removeRow(oldRow.link()); - } + dataTree.invoke(clo, CacheDataRowAdapter.RowData.LINK_ONLY, clo); - if (pendingTree() != null && expireTime != 0) { - pendingTree().putx(new PendingRow(cacheId, expireTime, newRow.link())); + if (clo.operationType() == PUT) + finishUpdate(cctx, clo, null); - hasPendingEntries = true; + return clo.operationType() == PUT; + } + finally { + busyLock.leaveBusy(); } - - updateIgfsMetrics(cctx, key, (oldRow != null ? oldRow.value() : null), newRow.value()); } /** {@inheritDoc} */ - @Override public void remove(GridCacheContext cctx, KeyCacheObject key, int partId) throws IgniteCheckedException { + @Override public boolean mvccUpdateRowWithPreloadInfo( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState) throws IgniteCheckedException { if (!busyLock.enterBusy()) throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); try { - int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + CacheObjectContext coCtx = cctx.cacheObjectContext(); - assert cctx.shared().database().checkpointLockIsHeldByThread(); + // Make sure value bytes initialized. + key.valueBytes(coCtx); - CacheDataRow oldRow = dataTree.remove(new SearchRow(cacheId, key)); + if (val != null) + val.valueBytes(coCtx); - finishRemove(cctx, key, oldRow); + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + MvccUpdateRowWithPreloadInfoClosure clo = new MvccUpdateRowWithPreloadInfoClosure(cctx, + key, + val, + ver, + expireTime, + mvccVer, + newMvccVer, + mvccTxState, + newMvccTxState); + + invoke0(cctx, clo, clo); } finally { busyLock.leaveBusy(); } + + return true; } - /** - * @param cctx Cache context. - * @param key Key. - * @param oldRow Removed row. - * @throws IgniteCheckedException If failed. - */ - private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable CacheDataRow oldRow) throws IgniteCheckedException { - if (oldRow != null) { + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccUpdate( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory, + boolean noCreate) throws IgniteCheckedException { + assert mvccSnapshot != null; + assert primary || !needHistory; + + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - assert oldRow.link() != 0 : oldRow; - assert cacheId == CU.UNDEFINED_CACHE_ID || oldRow.cacheId() == cacheId : - "Incorrect cache ID [expected=" + cacheId + ", actual=" + oldRow.cacheId() + "]."; + CacheObjectContext coCtx = cctx.cacheObjectContext(); - if (pendingTree() != null && oldRow.expireTime() != 0) - pendingTree().removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); + // Make sure value bytes initialized. + key.valueBytes(coCtx); + val.valueBytes(coCtx); - decrementSize(cctx.cacheId()); - } + MvccUpdateDataRow updateRow = new MvccUpdateDataRow( + cctx, + key, + val, + ver, + partId, + expireTime, + mvccSnapshot, + null, + primary, + false, + needHistory, + // we follow fast update visit flow here if row cannot be created by current operation + noCreate); - GridCacheQueryManager qryMgr = cctx.queries(); + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + dataTree.visit(new MvccMaxSearchRow(cacheId, key), new MvccMinSearchRow(cacheId, key), updateRow); + + ResultType res = updateRow.resultType(); + + if (res == ResultType.LOCKED // cannot update locked + || res == ResultType.VERSION_MISMATCH) // cannot update on write conflict + return updateRow; + else if (res == ResultType.VERSION_FOUND) { + // Do nothing, except cleaning up not needed versions + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow; + } + else if (res == ResultType.PREV_NULL && noCreate) { + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow; + } + + CacheDataRow oldRow = null; + + if (res == ResultType.PREV_NOT_NULL) { + oldRow = updateRow.oldRow(); + + assert oldRow != null && oldRow.link() != 0 : oldRow; + + oldRow.key(key); + + rowStore.updateDataRow(oldRow.link(), mvccUpdateMarker, mvccSnapshot); + } + else + assert res == ResultType.PREV_NULL; + + if (!grp.storeCacheIdInDataPage() && updateRow.cacheId() != CU.UNDEFINED_CACHE_ID) { + updateRow.cacheId(CU.UNDEFINED_CACHE_ID); + + rowStore.addRow(updateRow); + + updateRow.cacheId(cctx.cacheId()); + } + else + rowStore.addRow(updateRow); + + if (needHistory) { + assert updateRow.link() != 0; + + updateRow.history().add(new MvccLinkAwareSearchRow(cacheId, + key, + updateRow.mvccCoordinatorVersion(), + updateRow.mvccCounter(), + updateRow.mvccOperationCounter(), + updateRow.link())); + } + + boolean old = dataTree.putx(updateRow); + + assert !old; + + incrementSize(cctx.cacheId()); + + GridCacheQueryManager qryMgr = cctx.queries(); + + if (qryMgr.enabled()) + qryMgr.store(updateRow, null, true); + + updatePendingEntries(cctx, updateRow, oldRow); + + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow; + } + finally { + busyLock.leaveBusy(); + } + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccRemove(GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot mvccSnapshot, + boolean primary, + boolean needHistory) throws IgniteCheckedException { + assert mvccSnapshot != null; + assert primary || mvccSnapshot.activeTransactions().size() == 0 : mvccSnapshot; + assert primary || !needHistory; + + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + // Make sure value bytes initialized. + key.valueBytes(coCtx); + + MvccUpdateDataRow updateRow = new MvccUpdateDataRow( + cctx, + key, + null, + null, + partId, + 0, + mvccSnapshot, + null, + primary, + false, + needHistory, + true); + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + dataTree.visit(new MvccMaxSearchRow(cacheId, key), new MvccMinSearchRow(cacheId, key), updateRow); + + ResultType res = updateRow.resultType(); + + if (res == ResultType.LOCKED // cannot update locked + || res == ResultType.VERSION_MISMATCH) // cannot update on write conflict + return updateRow; + else if (res == ResultType.VERSION_FOUND) { + // Do nothing, except cleaning up not needed versions + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow; + } + else if (res == ResultType.PREV_NOT_NULL) { + CacheDataRow oldRow = updateRow.oldRow(); + + assert oldRow != null && oldRow.link() != 0 : oldRow; + + rowStore.updateDataRow(oldRow.link(), mvccUpdateMarker, mvccSnapshot); + + clearPendingEntries(cctx, oldRow); + } + + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow; + } + finally { + busyLock.leaveBusy(); + } + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccLock(GridCacheContext cctx, KeyCacheObject key, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + assert mvccSnapshot != null; + + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + // Make sure value bytes initialized. + key.valueBytes(coCtx); + + MvccUpdateDataRow updateRow = new MvccUpdateDataRow( + cctx, + key, + null, + null, + partId, + 0, + mvccSnapshot, + null, + true, + true, + false, + false); + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + dataTree.visit(new MvccMaxSearchRow(cacheId, key), new MvccMinSearchRow(cacheId, key), updateRow); + + ResultType res = updateRow.resultType(); + + // cannot update locked, cannot update on write conflict + if (res == ResultType.LOCKED || res == ResultType.VERSION_MISMATCH) + return updateRow; + + // Do nothing, except cleaning up not needed versions + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow; + } + finally { + busyLock.leaveBusy(); + } + } + + /** {@inheritDoc} */ + @Override public GridLongList mvccUpdateNative( + GridCacheContext cctx, + boolean primary, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + assert mvccSnapshot != null; + assert primary || mvccSnapshot.activeTransactions().size() == 0 : mvccSnapshot; + + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + // Make sure value bytes initialized. + key.valueBytes(coCtx); + val.valueBytes(coCtx); + + MvccUpdateDataRowNative updateRow = new MvccUpdateDataRowNative( + key, + val, + ver, + expireTime, + mvccSnapshot, + null, + partId, + cctx); + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + dataTree.iterate(new MvccMaxSearchRow(cacheId, key), new MvccMinSearchRow(cacheId, key), updateRow); + + ResultType res = updateRow.resultType(); + + if (res == ResultType.VERSION_FOUND) { + // Do nothing, except cleaning up not needed versions + cleanup(cctx, updateRow.cleanupRows()); + + return null; + } + + CacheDataRow oldRow = null; + + if (res == ResultType.PREV_NOT_NULL) { + oldRow = updateRow.oldRow(); + + assert oldRow != null && oldRow.link() != 0 : oldRow; + + oldRow.key(key); + + rowStore.updateDataRow(oldRow.link(), mvccUpdateMarker, mvccSnapshot); + } + else + assert res == ResultType.PREV_NULL; + + if (!grp.storeCacheIdInDataPage() && updateRow.cacheId() != CU.UNDEFINED_CACHE_ID) { + updateRow.cacheId(CU.UNDEFINED_CACHE_ID); + + rowStore.addRow(updateRow); + + updateRow.cacheId(cctx.cacheId()); + } + else + rowStore.addRow(updateRow); + + boolean old = dataTree.putx(updateRow); + + assert !old; + + incrementSize(cctx.cacheId()); + + GridCacheQueryManager qryMgr = cctx.queries(); + + if (qryMgr.enabled()) + qryMgr.store(updateRow, null, true); + + updatePendingEntries(cctx, updateRow, oldRow); + + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow.activeTransactions(); + } + finally { + busyLock.leaveBusy(); + } + } + + /** {@inheritDoc} */ + @Override public GridLongList mvccRemoveNative(GridCacheContext cctx, + boolean primary, + KeyCacheObject key, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + assert mvccSnapshot != null; + assert primary || mvccSnapshot.activeTransactions().size() == 0 : mvccSnapshot; + + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + // Make sure value bytes initialized. + key.valueBytes(coCtx); + + MvccUpdateDataRowNative updateRow = new MvccUpdateDataRowNative( + key, + null, + null, + 0, + mvccSnapshot, + null, + partId, + cctx); + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + dataTree.iterate(new MvccMaxSearchRow(cacheId, key) , new MvccMinSearchRow(cacheId, key), updateRow); + + ResultType res = updateRow.resultType(); + + if (res == ResultType.VERSION_FOUND) { + assert !primary : updateRow; + + // Do nothing, except cleaning up not needed versions + cleanup(cctx, updateRow.cleanupRows()); + + return null; + } + else if (res == ResultType.PREV_NOT_NULL) { + CacheDataRow oldRow = updateRow.oldRow(); + + assert oldRow != null && oldRow.link() != 0 : oldRow; + + rowStore.updateDataRow(oldRow.link(), mvccUpdateMarker, mvccSnapshot); + + clearPendingEntries(cctx, oldRow); + } + + cleanup(cctx, updateRow.cleanupRows()); + + return updateRow.activeTransactions(); + } + finally { + busyLock.leaveBusy(); + } + } + + /** {@inheritDoc} */ + @Override public void mvccRemoveAll(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { + key.valueBytes(cctx.cacheObjectContext()); + + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + boolean cleanup = cctx.queries().enabled() || hasPendingEntries; + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + GridCursor cur = dataTree.find( + new MvccMaxSearchRow(cacheId, key), + new MvccMinSearchRow(cacheId, key), + cleanup ? CacheDataRowAdapter.RowData.NO_KEY : CacheDataRowAdapter.RowData.LINK_ONLY + ); + + boolean first = true; + + while (cur.next()) { + CacheDataRow row = cur.get(); + + row.key(key); + + assert row.link() != 0 : row; + + boolean rmvd = dataTree.removex(row); + + assert rmvd : row; + + if (cleanup) { + if (cctx.queries().enabled()) + cctx.queries().remove(key, row); + + if (first) + clearPendingEntries(cctx, row); + } + + rowStore.removeRow(row.link()); + + decrementSize(cctx.cacheId()); + + if (first) + first = false; + } + } + + /** {@inheritDoc} */ + @Override public int cleanup(GridCacheContext cctx, @Nullable List cleanupRows) + throws IgniteCheckedException { + int res = 0; + + if (cleanupRows != null) { + GridCacheQueryManager qryMgr = cctx.queries(); + + for (int i = 0; i < cleanupRows.size(); i++) { + MvccLinkAwareSearchRow cleanupRow = cleanupRows.get(i); + + assert cleanupRow.link() != 0 : cleanupRow; + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + CacheDataRow oldRow = dataTree.remove(cleanupRow); + + if (oldRow != null) { // oldRow == null means it was cleaned by another cleanup process. + assert oldRow.mvccCounter() == cleanupRow.mvccCounter(); + + if (qryMgr.enabled()) + qryMgr.remove(oldRow.key(), oldRow); + + clearPendingEntries(cctx, oldRow); + + rowStore.removeRow(cleanupRow.link()); + + decrementSize(cctx.cacheId()); + + res++; + } + } + } + + return res; + } + + /** {@inheritDoc} */ + @Override public void updateTxState(GridCacheContext cctx, CacheSearchRow row) + throws IgniteCheckedException { + assert grp.mvccEnabled(); + assert mvccVersionIsValid(row.mvccCoordinatorVersion(), row.mvccCounter(), row.mvccOperationCounter()) : row; + + // Need an extra lookup because the row may be already cleaned by another thread. + CacheDataRow row0 = dataTree.findOne(row, CacheDataRowAdapter.RowData.LINK_ONLY); + + if (row0 != null) + rowStore.updateDataRow(row0.link(), mvccUpdateTxStateHint, null); + } + + /** {@inheritDoc} */ + @Override public void update(GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + @Nullable CacheDataRow oldRow + ) throws IgniteCheckedException { + assert oldRow == null || oldRow.link() != 0L : oldRow; + + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { + int cacheId = grp.storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + assert oldRow == null || oldRow.cacheId() == cacheId : oldRow; + + if (key.partition() == -1) + key.partition(partId); + + DataRow dataRow = new DataRow(key, val, ver, partId, expireTime, cacheId); + + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + // Make sure value bytes initialized. + key.valueBytes(coCtx); + val.valueBytes(coCtx); + + CacheDataRow old; + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + if (canUpdateOldRow(cctx, oldRow, dataRow) && rowStore.updateRow(oldRow.link(), dataRow)) { + old = oldRow; + + dataRow.link(oldRow.link()); + } + else { + rowStore.addRow(dataRow); + + assert dataRow.link() != 0 : dataRow; + + if (grp.sharedGroup() && dataRow.cacheId() == CU.UNDEFINED_CACHE_ID) + dataRow.cacheId(cctx.cacheId()); + + if (oldRow != null) { + old = oldRow; + + dataTree.putx(dataRow); + } + else + old = dataTree.put(dataRow); + } + + finishUpdate(cctx, dataRow, old); + } + finally { + busyLock.leaveBusy(); + } + } + + /** + * @param cctx Cache context. + * @param newRow New row. + * @param oldRow Old row if available. + * @throws IgniteCheckedException If failed. + */ + private void finishUpdate(GridCacheContext cctx, CacheDataRow newRow, @Nullable CacheDataRow oldRow) + throws IgniteCheckedException { + if (oldRow == null) + incrementSize(cctx.cacheId()); + + KeyCacheObject key = newRow.key(); + + GridCacheQueryManager qryMgr = cctx.queries(); + + if (qryMgr.enabled()) + qryMgr.store(newRow, oldRow, true); + + updatePendingEntries(cctx, newRow, oldRow); + + if (oldRow != null) { + assert oldRow.link() != 0 : oldRow; + + if (newRow.link() != oldRow.link()) + rowStore.removeRow(oldRow.link()); + } + + updateIgfsMetrics(cctx, key, (oldRow != null ? oldRow.value() : null), newRow.value()); + } + + /** + * @param cctx Cache context. + * @param newRow New row. + * @param oldRow Old row. + * @throws IgniteCheckedException If failed. + */ + private void updatePendingEntries(GridCacheContext cctx, CacheDataRow newRow, @Nullable CacheDataRow oldRow) + throws IgniteCheckedException + { + long expireTime = newRow.expireTime(); + + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + if (oldRow != null) { + assert oldRow.link() != 0 : oldRow; + + if (pendingTree() != null && oldRow.expireTime() != 0) + pendingTree().removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); + } + + if (pendingTree() != null && expireTime != 0) { + pendingTree().putx(new PendingRow(cacheId, expireTime, newRow.link())); + + hasPendingEntries = true; + } + } + + /** {@inheritDoc} */ + @Override public void remove(GridCacheContext cctx, KeyCacheObject key, int partId) throws IgniteCheckedException { + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + CacheDataRow oldRow = dataTree.remove(new SearchRow(cacheId, key)); + + finishRemove(cctx, key, oldRow); + } + finally { + busyLock.leaveBusy(); + } + } + + /** + * @param cctx Cache context. + * @param key Key. + * @param oldRow Removed row. + * @throws IgniteCheckedException If failed. + */ + private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable CacheDataRow oldRow) throws IgniteCheckedException { + if (oldRow != null) { + clearPendingEntries(cctx, oldRow); + + decrementSize(cctx.cacheId()); + } + + GridCacheQueryManager qryMgr = cctx.queries(); if (qryMgr.enabled()) qryMgr.remove(key, oldRow); @@ -1521,21 +2486,129 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C updateIgfsMetrics(cctx, key, (oldRow != null ? oldRow.value() : null), null); } + /** + * @param cctx Cache context. + * @param oldRow Old row. + * @throws IgniteCheckedException + */ + private void clearPendingEntries(GridCacheContext cctx, CacheDataRow oldRow) + throws IgniteCheckedException { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + assert oldRow.link() != 0 : oldRow; + assert cacheId == CU.UNDEFINED_CACHE_ID || oldRow.cacheId() == cacheId : + "Incorrect cache ID [expected=" + cacheId + ", actual=" + oldRow.cacheId() + "]."; + + if (pendingTree() != null && oldRow.expireTime() != 0) + pendingTree().removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); + } + /** {@inheritDoc} */ @Override public CacheDataRow find(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { key.valueBytes(cctx.cacheObjectContext()); int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - CacheDataRow row = dataTree.findOne(new SearchRow(cacheId, key), CacheDataRowAdapter.RowData.NO_KEY); + CacheDataRow row; + + if (grp.mvccEnabled()) { + MvccFirstRowTreeClosure clo = new MvccFirstRowTreeClosure(cctx); + + dataTree.iterate( + new MvccMaxSearchRow(cacheId, key), + new MvccMinSearchRow(cacheId, key), + clo + ); + + row = clo.row(); + } + else + row = dataTree.findOne(new SearchRow(cacheId, key), CacheDataRowAdapter.RowData.NO_KEY); + + afterRowFound(row, key); + + return row; + } + + /** {@inheritDoc} */ + @Override public List> mvccFindAllVersions( + GridCacheContext cctx, + KeyCacheObject key) + throws IgniteCheckedException + { + assert grp.mvccEnabled(); + + // Note: this method is intended for testing only. + + key.valueBytes(cctx.cacheObjectContext()); + + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + GridCursor cur = dataTree.find( + new MvccMaxSearchRow(cacheId, key), + new MvccMinSearchRow(cacheId, key) + ); + + List> res = new ArrayList<>(); + + long crd = MVCC_CRD_COUNTER_NA, cntr = MVCC_COUNTER_NA; int opCntr = MVCC_OP_COUNTER_NA; + + while (cur.next()) { + CacheDataRow row = cur.get(); + + if (compareNewVersion(row, crd, cntr, opCntr) != 0) // deleted row + res.add(F.t(null, row.newMvccVersion())); + + res.add(F.t(row.value(), row.mvccVersion())); + + crd = row.mvccCoordinatorVersion(); cntr = row.mvccCounter(); opCntr = row.mvccOperationCounter(); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public GridCursor mvccAllVersionsCursor(GridCacheContext cctx, KeyCacheObject key, Object x) + throws IgniteCheckedException { + int cacheId = cctx.cacheId(); + + return dataTree.find(new MvccMaxSearchRow(cacheId, key), new MvccMinSearchRow(cacheId, key), x); + } + + /** {@inheritDoc} */ + @Override public CacheDataRow mvccFind(GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot snapshot) throws IgniteCheckedException { + key.valueBytes(cctx.cacheObjectContext()); + + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + MvccSnapshotSearchRow clo = new MvccSnapshotSearchRow(cctx, key, snapshot); + + dataTree.iterate( + clo, + new MvccMinSearchRow(cacheId, key), + clo + ); + + CacheDataRow row = clo.row(); + + afterRowFound(row, key); + return row; + } + + /** + * @param row Row. + * @param key Key. + * @throws IgniteCheckedException If failed. + */ + private void afterRowFound(@Nullable CacheDataRow row, KeyCacheObject key) throws IgniteCheckedException { if (row != null) { row.key(key); grp.dataRegion().evictionTracker().touchPage(row.link()); } - - return row; } /** {@inheritDoc} */ @@ -1543,11 +2616,37 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C return dataTree.find(null, null); } + /** {@inheritDoc} */ + @Override public GridCursor cursor(Object x) throws IgniteCheckedException { + return dataTree.find(null, null, x); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(MvccSnapshot mvccSnapshot) + throws IgniteCheckedException { + + if (mvccSnapshot != null) { + assert grp.mvccEnabled(); + + return dataTree.find(null, null, + new MvccFirstVisibleRowTreeClosure(grp.singleCacheContext(), mvccSnapshot), null); + } + + return dataTree.find(null, null); + } + /** {@inheritDoc} */ @Override public GridCursor cursor(int cacheId) throws IgniteCheckedException { return cursor(cacheId, null, null); } + /** {@inheritDoc} + * @param cacheId*/ + @Override public GridCursor cursor(int cacheId, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + return cursor(cacheId, null, null, null, mvccSnapshot); + } + /** {@inheritDoc} */ @Override public GridCursor cursor(int cacheId, KeyCacheObject lower, KeyCacheObject upper) throws IgniteCheckedException { @@ -1557,6 +2656,12 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C /** {@inheritDoc} */ @Override public GridCursor cursor(int cacheId, KeyCacheObject lower, KeyCacheObject upper, Object x) throws IgniteCheckedException { + return cursor(cacheId, lower, upper, null, null); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId, KeyCacheObject lower, + KeyCacheObject upper, Object x, MvccSnapshot snapshot) throws IgniteCheckedException { SearchRow lowerRow; SearchRow upperRow; @@ -1571,6 +2676,15 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C upperRow = upper != null ? new SearchRow(CU.UNDEFINED_CACHE_ID, upper) : null; } + if (snapshot != null) { + assert grp.mvccEnabled(); + + GridCacheContext cctx = + grp.sharedGroup() ? grp.shared().cacheContext(cacheId) : grp.singleCacheContext(); + + return dataTree.find(lowerRow, upperRow, new MvccFirstVisibleRowTreeClosure(cctx, snapshot), x); + } + return dataTree.find(lowerRow, upperRow, x); } @@ -1645,13 +2759,6 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C return rowStore; } - /** - * @return Next update index. - */ - @Override public long nextUpdateCounter() { - return cntr.incrementAndGet(); - } - /** {@inheritDoc} */ @Override public long initialUpdateCounter() { return initCntr; @@ -1739,5 +2846,337 @@ private int valueLength(GridCacheContext cctx, @Nullable CacheObject val) { else return 0; } + + /** */ + private final class MvccFirstVisibleRowTreeClosure implements MvccTreeClosure { + /** */ + private final GridCacheContext cctx; + + /** */ + private final MvccSnapshot snapshot; + + /** + * + * @param cctx Cache context. + * @param snapshot MVCC snapshot. + */ + MvccFirstVisibleRowTreeClosure(GridCacheContext cctx, MvccSnapshot snapshot) { + this.cctx = cctx; + this.snapshot = snapshot; + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, + long pageAddr, int idx) throws IgniteCheckedException { + RowLinkIO rowIo = (RowLinkIO)io; + + long rowCrdVer = rowIo.getMvccCoordinatorVersion(pageAddr, idx); + long rowCntr = rowIo.getMvccCounter(pageAddr, idx); + int rowOpCntr = rowIo.getMvccOperationCounter(pageAddr, idx); + + assert mvccVersionIsValid(rowCrdVer, rowCntr, rowOpCntr); + + return isVisible(cctx, snapshot, rowCrdVer, rowCntr, rowOpCntr, rowIo.getLink(pageAddr, idx)); + } + } + + /** + * Put row if it doesn't exist yet. + */ + private class MvccPutIfAbsentClosure extends MvccDataRow implements IgniteTree.InvokeClosure { + /** */ + private IgniteTree.OperationType op; + + /** + * @param key Key. + * @param val Value. + * @param ver Version. + * @param part Partition. + * @param expireTime Expire time. + * @param cacheId Cache ID. + * @param mvccVer Mvcc version. + * @param newMvccVer New mvcc version. + * @param txState Tx state hint for mvcc version. + * @param newTxState Tx state hint for new mvcc version. + */ + MvccPutIfAbsentClosure(KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + int part, + long expireTime, + int cacheId, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte txState, + byte newTxState) { + super(key, val, ver, part, expireTime, cacheId, mvccVer, newMvccVer); + + mvccTxState(txState); + newMvccTxState(newTxState); + } + + /** {@inheritDoc} */ + @Override public void call(@Nullable CacheDataRow old) throws IgniteCheckedException { + if (old == null) { + op = PUT; + + int cacheId = cacheId(); + + if (!grp.storeCacheIdInDataPage() && cacheId != CU.UNDEFINED_CACHE_ID) + cacheId(CU.UNDEFINED_CACHE_ID); + + rowStore().addRow(this); + + cacheId(cacheId); + } + else + op = NOOP; + } + + /** {@inheritDoc} */ + @Override public MvccDataRow newRow() { + return this; + } + + /** {@inheritDoc} */ + @Override public IgniteTree.OperationType operationType() { + return op; + } + } + + /** + * + */ + private class MvccUpdateRowWithPreloadInfoClosure extends MvccDataRow implements OffheapInvokeClosure { + /** */ + private CacheDataRow oldRow; + /** */ + private IgniteTree.OperationType op; + + /** + * @param cctx Cache context. + * @param key Key. + * @param val Value. + * @param ver Version. + * @param expireTime Expire time. + * @param mvccVer Mvcc created version. + * @param newMvccVer Mvcc updated version. + * @param mvccTxState Mvcc Tx state hint. + * @param newMvccTxState New Mvcc Tx state hint. + */ + MvccUpdateRowWithPreloadInfoClosure(GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState) { + super(key, + val, + ver, + CacheDataStoreImpl.this.partId(), + expireTime, + cctx.cacheId(), + mvccVer, + newMvccVer); + + mvccTxState(mvccTxState); + newMvccTxState(newMvccTxState); + } + + /** {@inheritDoc} */ + @Nullable @Override public CacheDataRow oldRow() { + return oldRow; + } + + /** {@inheritDoc} */ + @Override public void call(@Nullable CacheDataRow oldRow) throws IgniteCheckedException { + this.oldRow = oldRow; + + if (oldRow == null) { + op = PUT; + + int cacheId = cacheId(); + + if (!grp.storeCacheIdInDataPage() && cacheId != CU.UNDEFINED_CACHE_ID) + cacheId(CU.UNDEFINED_CACHE_ID); + + rowStore().addRow(this); + + cacheId(cacheId); + } + else { + op = NOOP; + + if (oldRow.mvccTxState() != mvccTxState() || + oldRow.newMvccCoordinatorVersion() != newMvccCoordinatorVersion() || + oldRow.newMvccCounter() != newMvccCounter() || + oldRow.newMvccOperationCounter() != newMvccOperationCounter() || + oldRow.newMvccTxState() != newMvccTxState()) { + + rowStore().updateDataRow(oldRow.link(), mvccApplyChanges, this); + } + } + } + + /** {@inheritDoc} */ + @Override public CacheDataRow newRow() { + return op == PUT ? this : null; + } + + /** {@inheritDoc} */ + @Override public IgniteTree.OperationType operationType() { + return op == null ? NOOP : op; + } + } + } + + /** + * Mvcc remove handler. + */ + private final class MvccMarkUpdatedHandler extends PageHandler { + /** {@inheritDoc} */ + @Override public Boolean run(int cacheId, long pageId, long page, long pageAddr, PageIO io, Boolean walPlc, + MvccVersion newVer, int itemId) throws IgniteCheckedException { + assert grp.mvccEnabled(); + + DataPageIO iox = (DataPageIO)io; + + int offset = iox.getPayloadOffset(pageAddr, itemId, grp.dataRegion().pageMemory().pageSize(), MVCC_INFO_SIZE); + + long newCrd = iox.newMvccCoordinator(pageAddr, offset); + long newCntr = iox.newMvccCounter(pageAddr, offset); + int newOpCntr = iox.newMvccOperationCounter(pageAddr, offset); + + assert newCrd == MVCC_CRD_COUNTER_NA || state(grp, newCrd, newCntr, newOpCntr) == TxState.ABORTED; + + iox.updateNewVersion(pageAddr, offset, newVer, TxState.NA); + + if (isWalDeltaRecordNeeded(grp.dataRegion().pageMemory(), cacheId, pageId, page, ctx.wal(), walPlc)) + ctx.wal().log(new DataPageMvccMarkUpdatedRecord(cacheId, pageId, itemId, + newVer.coordinatorVersion(), newVer.counter(), newVer.operationCounter())); + + return Boolean.TRUE; + } + } + + /** + * Mvcc update operation counter hints handler. + */ + private final class MvccUpdateTxStateHintHandler extends PageHandler { + /** {@inheritDoc} */ + @Override public Boolean run(int cacheId, long pageId, long page, long pageAddr, PageIO io, + Boolean walPlc, Void ignore, + int itemId) throws IgniteCheckedException { + + DataPageIO iox = (DataPageIO)io; + + int pageSize = grp.dataRegion().pageMemory().pageSize(); + int offset = iox.getPayloadOffset(pageAddr, itemId, pageSize, MVCC_INFO_SIZE); + + long crd = iox.mvccCoordinator(pageAddr, offset); + long cntr = iox.mvccCounter(pageAddr, offset); + int opCntr = iox.mvccOperationCounter(pageAddr, offset); + byte txState = (byte)(opCntr >>> MVCC_HINTS_BIT_OFF); + + if (txState == TxState.NA) { + byte state = state(grp, crd, cntr, opCntr); + + if (state == TxState.COMMITTED || state == TxState.ABORTED) { + iox.mvccOperationCounter(pageAddr, offset, opCntr | (state << MVCC_HINTS_BIT_OFF)); + + if (isWalDeltaRecordNeeded(grp.dataRegion().pageMemory(), cacheId, pageId, page, ctx.wal(), walPlc)) + ctx.wal().log(new DataPageMvccUpdateTxStateHintRecord(cacheId, pageId, itemId, state)); + } + else + throw unexpectedStateException(grp, state, crd, cntr, opCntr); + } + + long newCrd = iox.newMvccCoordinator(pageAddr, offset); + long newCntr = iox.newMvccCounter(pageAddr, offset); + int newOpCntr = iox.newMvccOperationCounter(pageAddr, offset); + byte newTxState = (byte)(newOpCntr >>> MVCC_HINTS_BIT_OFF); + + if (newCrd != MVCC_CRD_COUNTER_NA && newTxState == TxState.NA) { + byte state = state(grp, newCrd, newCntr, newOpCntr); + + if (state == TxState.COMMITTED || state == TxState.ABORTED) { + iox.newMvccOperationCounter(pageAddr, offset, newOpCntr | (state << MVCC_HINTS_BIT_OFF)); + + if (isWalDeltaRecordNeeded(grp.dataRegion().pageMemory(), cacheId, pageId, page, ctx.wal(), walPlc)) + ctx.wal().log(new DataPageMvccUpdateNewTxStateHintRecord(cacheId, pageId, itemId, state)); + } + + // We do not throw an exception here because new version may be updated by active Tx at this moment. + } + + return Boolean.TRUE; + } + } + + /** + * Applies changes to the row. + */ + private final class MvccApplyChangesHandler extends PageHandler { + /** {@inheritDoc} */ + @Override public Boolean run(int cacheId, long pageId, long page, long pageAddr, PageIO io, Boolean walPlc, + MvccDataRow newRow, int itemId) throws IgniteCheckedException { + assert grp.mvccEnabled(); + + DataPageIO iox = (DataPageIO)io; + + int offset = iox.getPayloadOffset(pageAddr, itemId, grp.dataRegion().pageMemory().pageSize(), MVCC_INFO_SIZE); + + long crd = iox.mvccCoordinator(pageAddr, offset); + long cntr = iox.mvccCounter(pageAddr, offset); + int opCntrAndHint = iox.mvccOperationCounter(pageAddr, offset); + int opCntr = opCntrAndHint & ~MVCC_HINTS_MASK; + byte txState = (byte)(opCntrAndHint >>> MVCC_HINTS_BIT_OFF); + + long newCrd = iox.newMvccCoordinator(pageAddr, offset); + long newCntr = iox.newMvccCounter(pageAddr, offset); + int newOpCntrAndHint = iox.newMvccOperationCounter(pageAddr, offset); + int newOpCntr = newOpCntrAndHint & ~MVCC_HINTS_MASK; + byte newTxState = (byte)(newOpCntrAndHint >>> MVCC_HINTS_BIT_OFF); + + assert crd == newRow.mvccCoordinatorVersion(); + assert cntr == newRow.mvccCounter(); + assert opCntr == newRow.mvccOperationCounter(); + + if (txState != newRow.mvccTxState() && newRow.mvccTxState() != TxState.NA) { + assert txState == TxState.NA; + + iox.mvccOperationCounter(pageAddr, offset, opCntr | (newRow.mvccTxState() << MVCC_HINTS_BIT_OFF)); + + if (isWalDeltaRecordNeeded(grp.dataRegion().pageMemory(), cacheId, pageId, page, ctx.wal(), walPlc)) + ctx.wal().log(new DataPageMvccUpdateTxStateHintRecord(cacheId, pageId, itemId, newRow.mvccTxState())); + } + + if (compare(newCrd, + newCntr, + newOpCntr, + newRow.newMvccCoordinatorVersion(), + newRow.newMvccCounter(), + newRow.newMvccOperationCounter()) != 0) { + + iox.updateNewVersion(pageAddr, offset, newRow.newMvccVersion(), newRow.newMvccTxState()); + + if (isWalDeltaRecordNeeded(grp.dataRegion().pageMemory(), cacheId, pageId, page, ctx.wal(), walPlc)) + ctx.wal().log(new DataPageMvccMarkUpdatedRecord(cacheId, pageId, itemId, + newRow.newMvccCoordinatorVersion(), newRow.newMvccCounter(), newRow.newMvccOperationCounter())); + } + else if (newTxState != newRow.newMvccTxState() && newRow.newMvccTxState() != TxState.NA) { + assert newTxState == TxState.NA; + + iox.newMvccOperationCounter(pageAddr, offset, newOpCntr | (newRow.newMvccTxState() << MVCC_HINTS_BIT_OFF)); + + if (isWalDeltaRecordNeeded(grp.dataRegion().pageMemory(), cacheId, pageId, page, ctx.wal(), walPlc)) + ctx.wal().log(new DataPageMvccUpdateNewTxStateHintRecord(cacheId, pageId, itemId, newRow.newMvccTxState())); + } + + return Boolean.TRUE; + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java index e34f22f9dd54a..c21ad0bb87015 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java @@ -74,6 +74,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; @@ -347,6 +348,9 @@ public IgniteCacheProxy gatewayWrapper() { /** {@inheritDoc} */ @Override public Lock lockAll(final Collection keys) { + //TODO IGNITE-7764 + MvccUtils.verifyMvccOperationSupport(ctx, "Lock"); + return new CacheLockImpl<>(ctx.gate(), delegate, ctx.operationContextPerCall(), keys); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java index 7cc368a999b66..1f688f64cc9e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java @@ -248,7 +248,7 @@ public void onSend() { log.trace("Moved transaction synchronizer to waiting state [nodeId=" + nodeId + ", threadId=" + threadId + ']'); - assert cnt == 0 || nodeLeft; + assert cnt == 0 || nodeLeft : cnt; if (nodeLeft) return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java index 3eeb0dbe45fcb..41e8abae60b61 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxRecoveryFuture.java @@ -146,6 +146,17 @@ else if (log.isInfoEnabled()) */ @SuppressWarnings("ConstantConditions") public void prepare() { + if (tx.txState().mvccEnabled(cctx)) { // TODO IGNITE-7313 + U.error(log, "Cannot commit MVCC enabled transaction by recovery procedure. " + + "Operation is usupported at the moment [tx=" + CU.txString(tx) + ']'); + + onDone(false); + + markInitialized(); + + return; + } + if (nearTxCheck) { UUID nearNodeId = tx.eventNodeId(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java index 481c95486a278..b5437869c6594 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java @@ -46,6 +46,9 @@ public class GridDistributedTxMapping { /** Explicit lock flag. */ private boolean explicitLock; + /** Query update flag. */ + private boolean queryUpdate; + /** DHT version. */ private GridCacheVersion dhtVer; @@ -132,6 +135,20 @@ public Collection entries() { return F.view(entries, CU.FILTER_NEAR_CACHE_ENTRY); } + /** + * @return {@code True} if mapping was created for a query update. + */ + public boolean queryUpdate() { + return queryUpdate; + } + + /** + * Sets query update flag to {@code true}. + */ + public void markQueryUpdate() { + queryUpdate = true; + } + /** * @return {@code True} if lock is explicit. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java index 91dcd9e621de9..a5aa0d838935a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java @@ -78,6 +78,9 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage /** */ public static final int STORE_WRITE_THROUGH_FLAG_MASK = 0x20; + /** */ + public static final int QUERY_UPDATE_FLAG_MASK = 0x40; + /** Collection to message converter. */ private static final C1, UUIDCollectionMessage> COL_TO_MSG = new C1, UUIDCollectionMessage>() { @Override public UUIDCollectionMessage apply(Collection uuids) { @@ -252,6 +255,25 @@ public void storeWriteThrough(boolean storeWriteThrough) { flags &= ~STORE_WRITE_THROUGH_FLAG_MASK; } + /** + * + * @return Flag indicating whether it is a query update. + */ + public boolean queryUpdate() { + return (flags & QUERY_UPDATE_FLAG_MASK) != 0; + } + + /** + * + * @param queryUpdate Query update value. + */ + public void queryUpdate(boolean queryUpdate) { + if (queryUpdate) + flags = (byte)(flags | QUERY_UPDATE_FLAG_MASK); + else + flags &= ~QUERY_UPDATE_FLAG_MASK; + } + /** * @return IO policy. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index c1293fc0772bd..dd6ea48e1b33c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -50,6 +50,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; @@ -205,6 +207,16 @@ public GridDistributedTxRemoteAdapter( return false; } + /** {@inheritDoc} */ + @Override public void activeCachesDeploymentEnabled(boolean depEnabled) { + throw new UnsupportedOperationException("Remote tx doesn't support deployment."); + } + + /** {@inheritDoc} */ + @Override public void addActiveCache(GridCacheContext cacheCtx, boolean recovery) throws IgniteCheckedException { + txState.addActiveCache(cacheCtx, recovery, this); + } + /** * @return Checks if transaction has no entries. */ @@ -460,7 +472,7 @@ private void commitIfLocked() throws IgniteCheckedException { GridCacheReturnCompletableWrapper wrapper = null; - if (!F.isEmpty(writeMap)) { + if (!F.isEmpty(writeMap) || mvccSnapshot != null) { GridCacheReturn ret = null; if (!near() && !local() && onePhaseCommit()) { @@ -488,6 +500,8 @@ private void commitIfLocked() throws IgniteCheckedException { cctx.database().checkpointReadLock(); try { + assert !txState.mvccEnabled(cctx) || mvccSnapshot != null : "Mvcc is not initialized: " + this; + Collection entries = near() || cctx.snapshot().needTxReadLogging() ? allEntries() : writeEntries(); // Data entry to write to WAL and associated with it TxEntry. @@ -613,7 +627,8 @@ else if (conflictCtx.isMerge()) { CU.subjectId(this, cctx), resolveTaskName(), dhtVer, - txEntry.updateCounter()); + txEntry.updateCounter(), + mvccSnapshot()); else { assert val != null : txEntry; @@ -637,9 +652,10 @@ else if (conflictCtx.isMerge()) { CU.subjectId(this, cctx), resolveTaskName(), dhtVer, - txEntry.updateCounter()); + txEntry.updateCounter(), + mvccSnapshot()); - txEntry.updateCounter(updRes.updatePartitionCounter()); + txEntry.updateCounter(updRes.updateCounter()); if (updRes.loggedPointer() != null) ptr = updRes.loggedPointer(); @@ -674,9 +690,10 @@ else if (op == DELETE) { CU.subjectId(this, cctx), resolveTaskName(), dhtVer, - txEntry.updateCounter()); + txEntry.updateCounter(), + mvccSnapshot()); - txEntry.updateCounter(updRes.updatePartitionCounter()); + txEntry.updateCounter(updRes.updateCounter()); if (updRes.loggedPointer() != null) ptr = updRes.loggedPointer(); @@ -790,6 +807,8 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in } } + updateLocalCounters(); + if (!near() && !F.isEmpty(dataEntries) && cctx.wal() != null) { // Set new update counters for data entries received from persisted tx entries. List entriesWithCounters = dataEntries.stream() @@ -803,6 +822,8 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in cctx.wal().flush(ptr, false); } catch (StorageException e) { + err = e; + throw new IgniteCheckedException("Failed to log transaction record " + "(transaction will be rolled back): " + this, e); } @@ -810,6 +831,8 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in finally { cctx.database().checkpointReadUnlock(); + notifyDrManager(state() == COMMITTING && err == null); + if (wrapper != null) wrapper.initialize(ret); } @@ -822,6 +845,33 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in } } + /** + * Applies update counters to the local partitions. + */ + private void updateLocalCounters() { + Map updCntrsMap = updateCountersMap(); + + if (F.isEmpty(updCntrsMap)) + return; + + for (Map.Entry entry : updCntrsMap.entrySet()) { + GridCacheContext cacheCtx = cctx.cacheContext(entry.getKey()); + + GridDhtPartitionsUpdateCountersMap cacheUpdCntrs = entry.getValue(); + + assert cacheUpdCntrs != null && !F.isEmpty(cacheUpdCntrs.updateCounters()); + + for (Map.Entry e : cacheUpdCntrs.updateCounters().entrySet()) { + Long updCntr = e.getValue(); + GridDhtLocalPartition part = cacheCtx.topology().localPartition(e.getKey()); + + assert part != null && updCntr != null && updCntr > 0; + + part.updateCounter(updCntr); + } + } + } + /** {@inheritDoc} */ @Override public final void commitRemoteTx() throws IgniteCheckedException { if (optimistic()) @@ -901,6 +951,8 @@ public void forceCommit() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public final void rollbackRemoteTx() { try { + notifyDrManager(false); + // Note that we don't evict near entries here - // they will be deleted by their corresponding transactions. if (state(ROLLING_BACK) || state() == UNKNOWN) { @@ -970,9 +1022,18 @@ protected void addExplicit(IgniteTxEntry e) { } } + /** {@inheritDoc} */ + @Override public void updateCountersMap(Map updCntrsMap) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public Map updateCountersMap() { + return null; + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(GridDistributedTxRemoteAdapter.class, this, "super", super.toString()); } - } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CompoundLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CompoundLockFuture.java new file mode 100644 index 0000000000000..63eff40f82959 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CompoundLockFuture.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.distributed.dht; + +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxAbstractEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxSelectForUpdateFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.lang.IgniteInClosure; + +/** + * + */ +public class CompoundLockFuture extends GridFutureAdapter implements DhtLockFuture, IgniteInClosure> { + /** */ + private static final long serialVersionUID = 4644646033267042131L; + /** */ + private static final AtomicIntegerFieldUpdater CNT_UPD = + AtomicIntegerFieldUpdater.newUpdater(CompoundLockFuture.class, "cnt"); + /** */ + private volatile int cnt; + /** */ + private final GridDhtTxLocalAdapter tx; + + /** + * @param cnt ResultSet futures count. + * @param tx Transaction. + */ + public CompoundLockFuture(int cnt, GridDhtTxLocalAdapter tx) { + this.cnt = cnt; + this.tx = tx; + } + + /** + * @param fut ResultSet future. + */ + public void register(IgniteInternalFuture fut) { + fut.listen(this); + } + + /** + * Init method. + */ + public void init() { + while(true) { + IgniteInternalFuture fut = tx.lockFuture(); + + if (fut == GridDhtTxLocalAdapter.ROLLBACK_FUT) { + onDone(tx.timedOut() ? tx.timeoutException() : tx.rollbackException()); + + break; + } + else if (fut != null) { + // Wait for previous future. + assert fut instanceof GridNearTxAbstractEnlistFuture + || fut instanceof GridDhtTxAbstractEnlistFuture + || fut instanceof CompoundLockFuture + || fut instanceof GridNearTxSelectForUpdateFuture : fut; + + // Terminate this future if parent future is terminated by rollback. + if (!fut.isDone()) { + fut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + if (fut.error() != null) + onDone(fut.error()); + } + }); + } + else if (fut.error() != null) + onDone(fut.error()); + + break; + } + else if (tx.updateLockFuture(null, this)) + break; + } + } + + @Override public void apply(IgniteInternalFuture future) { + if (!isDone() && (future.error() != null || CNT_UPD.decrementAndGet(this) == 0)) { + Throwable err = future.error(); + + if (err == null) + tx.clearLockFuture(this); + + onDone(err); + } + } + + /** {@inheritDoc} */ + @Override public void onError(Throwable error) { + assert error != null; + + onDone(error); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/DhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/DhtLockFuture.java new file mode 100644 index 0000000000000..b729dcd7f788f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/DhtLockFuture.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht; + +import org.apache.ignite.internal.IgniteInternalFuture; + +/** + * Marker interface. + */ +public interface DhtLockFuture extends IgniteInternalFuture { + /** + * @param error Error. + */ + public void onError(Throwable error); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ExceptionAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ExceptionAware.java new file mode 100644 index 0000000000000..036492fc7c2d4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/ExceptionAware.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht; + +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public interface ExceptionAware { + /** + * @return Exception. + */ + @Nullable Throwable error(); +} 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 54a850c124510..70ce0821ae933 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 @@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.GridPartitionStateMap; @@ -198,6 +199,11 @@ private String mapString(GridDhtPartitionMap map) { lock.readLock().unlock(); } + /** {@inheritDoc} */ + @Override public MvccCoordinator mvccCoordinator() { + throw new UnsupportedOperationException(); + } + /** {@inheritDoc} */ @Override public boolean holdsLock() { return lock.isWriteLockedByCurrentThread() || lock.getReadHoldCount() > 0; @@ -207,6 +213,7 @@ private String mapString(GridDhtPartitionMap map) { @Override public void updateTopologyVersion( GridDhtTopologyFuture exchFut, DiscoCache discoCache, + MvccCoordinator mvccCrd, long updSeq, boolean stopping ) throws IgniteInterruptedCheckedException { 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 5adbda750d77d..cd37650c671b8 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 @@ -68,6 +68,8 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -230,16 +232,30 @@ private void processForceKeysRequest0(ClusterNode node, GridDhtForceKeysRequest entry.unswap(); - GridCacheEntryInfo info = entry.info(); + if (ctx.mvccEnabled()) { + List infos = entry.allVersionsInfo(); - if (info == null) { - assert entry.obsolete() : entry; + if (infos == null) { + assert entry.obsolete() : entry; - continue; + continue; + } + + for (int i = 0; i < infos.size(); i++) + res.addInfo(infos.get(i)); } + else { + GridCacheEntryInfo info = entry.info(); + + if (info == null) { + assert entry.obsolete() : entry; - if (!info.isNew()) - res.addInfo(info); + continue; + } + + if (!info.isNew()) + res.addInfo(info); + } entry.touch(msg.topologyVersion()); @@ -588,6 +604,9 @@ protected GridDistributedCacheEntry createEntry(KeyCacheObject key) { return; } + //TODO IGNITE-7954 + MvccUtils.verifyMvccOperationSupport(ctx, "Load"); + final AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion(); // Version for all loaded entries. @@ -766,6 +785,7 @@ else if (log.isDebugEnabled()) * @param taskName Task name. * @param expiry Expiry policy. * @param skipVals Skip values flag. + * @param mvccSnapshot MVCC snapshot. * @return Get future. */ IgniteInternalFuture> getDhtAllAsync( @@ -776,7 +796,8 @@ IgniteInternalFuture> getDhtAllAsync( String taskName, @Nullable IgniteCacheExpiryPolicy expiry, boolean skipVals, - boolean recovery + boolean recovery, + MvccSnapshot mvccSnapshot ) { return getAllAsync0(keys, readerArgs, @@ -789,7 +810,8 @@ IgniteInternalFuture> getDhtAllAsync( skipVals, /*keep cache objects*/true, recovery, - /*need version*/true); + /*need version*/true, + mvccSnapshot); } /** @@ -803,6 +825,7 @@ IgniteInternalFuture> getDhtAllAsync( * @param taskNameHash Task name hash code. * @param expiry Expiry policy. * @param skipVals Skip values flag. + * @param mvccSnapshot MVCC snapshot. * @return DHT future. */ public GridDhtFuture> getDhtAsync(UUID reader, @@ -815,7 +838,8 @@ public GridDhtFuture> getDhtAsync(UUID reader, int taskNameHash, @Nullable IgniteCacheExpiryPolicy expiry, boolean skipVals, - boolean recovery + boolean recovery, + MvccSnapshot mvccSnapshot ) { GridDhtGetFuture fut = new GridDhtGetFuture<>(ctx, msgId, @@ -828,7 +852,8 @@ public GridDhtFuture> getDhtAsync(UUID reader, expiry, skipVals, recovery, - addReaders); + addReaders, + mvccSnapshot); fut.init(); @@ -846,9 +871,10 @@ public GridDhtFuture> getDhtAsync(UUID reader, * @param taskNameHash Task name hash. * @param expiry Expiry. * @param skipVals Skip vals flag. + * @param mvccSnapshot Mvcc snapshot. * @return Future for the operation. */ - public GridDhtGetSingleFuture getDhtSingleAsync( + GridDhtGetSingleFuture getDhtSingleAsync( UUID nodeId, long msgId, KeyCacheObject key, @@ -859,7 +885,8 @@ public GridDhtGetSingleFuture getDhtSingleAsync( int taskNameHash, @Nullable IgniteCacheExpiryPolicy expiry, boolean skipVals, - boolean recovery + boolean recovery, + MvccSnapshot mvccSnapshot ) { GridDhtGetSingleFuture fut = new GridDhtGetSingleFuture<>( ctx, @@ -873,7 +900,8 @@ public GridDhtGetSingleFuture getDhtSingleAsync( taskNameHash, expiry, skipVals, - recovery); + recovery, + mvccSnapshot); fut.init(); @@ -901,7 +929,8 @@ protected void processNearSingleGetRequest(final UUID nodeId, final GridNearSing req.taskNameHash(), expiryPlc, req.skipValues(), - req.recovery()); + req.recovery(), + req.mvccSnapshot()); fut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture f) { @@ -1004,7 +1033,8 @@ protected void processNearGetRequest(final UUID nodeId, final GridNearGetRequest req.taskNameHash(), expiryPlc, req.skipValues(), - req.recovery()); + req.recovery(), + req.mvccSnapshot()); fut.listen(new CI1>>() { @Override public void apply(IgniteInternalFuture> f) { @@ -1230,6 +1260,8 @@ protected final boolean needRemap(AffinityTopologyVersion expVer, AffinityTopolo if (expVer.equals(curVer)) return false; + // TODO IGNITE-7164 check mvcc crd for mvcc enabled txs. + Collection cacheNodes0 = ctx.discovery().cacheGroupAffinityNodes(ctx.groupId(), expVer); Collection cacheNodes1 = ctx.discovery().cacheGroupAffinityNodes(ctx.groupId(), curVer); 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 fe02090a9c2c4..343e418891f85 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 @@ -101,6 +101,11 @@ public GridDhtCacheEntry( return locPart.nextUpdateCounter(cctx.cacheId(), topVer, primary, primaryCntr); } + /** {@inheritDoc} */ + @Override protected long nextMvccPartitionCounter() { + return locPart.nextMvccUpdateCounter(); + } + /** {@inheritDoc} */ @Override public int memorySize() throws IgniteCheckedException { int rdrsOverhead; @@ -648,7 +653,10 @@ public boolean clearInternal( ']'); } - removeValue(); + if (cctx.mvccEnabled()) + cctx.offheap().mvccRemoveAll(this); + else + removeValue(); // Give to GC. update(null, 0L, 0L, ver, true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java index a969dd32dab60..023c0583ed246 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java @@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.ReaderArguments; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture; @@ -114,6 +115,9 @@ public final class GridDhtGetFuture extends GridCompoundIdentityFuture extends GridCompoundIdentityFuture cctx, @@ -138,7 +143,8 @@ public GridDhtGetFuture( @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean skipVals, boolean recovery, - boolean addReaders + boolean addReaders, + MvccSnapshot mvccSnapshot ) { super(CU.collectionsReducer(keys.size())); @@ -157,6 +163,7 @@ public GridDhtGetFuture( this.skipVals = skipVals; this.recovery = recovery; this.addReaders = addReaders; + this.mvccSnapshot = mvccSnapshot; futId = IgniteUuid.randomUuid(); @@ -422,7 +429,8 @@ private IgniteInternalFuture> getAsync( taskName, expiryPlc, skipVals, - recovery); + recovery, + mvccSnapshot); } else { final ReaderArguments args = readerArgs; @@ -445,7 +453,8 @@ private IgniteInternalFuture> getAsync( taskName, expiryPlc, skipVals, - recovery); + recovery, + mvccSnapshot); } } ); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java index 99a8135cfc441..de76eb1c9ecf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.ReaderArguments; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; @@ -103,6 +104,9 @@ public final class GridDhtGetSingleFuture extends GridFutureAdapter extends GridFutureAdapter cctx, @@ -128,7 +133,8 @@ public GridDhtGetSingleFuture( int taskNameHash, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean skipVals, - boolean recovery + boolean recovery, + @Nullable MvccSnapshot mvccSnapshot ) { assert reader != null; assert key != null; @@ -145,6 +151,7 @@ public GridDhtGetSingleFuture( this.expiryPlc = expiryPlc; this.skipVals = skipVals; this.recovery = recovery; + this.mvccSnapshot = mvccSnapshot; futId = IgniteUuid.randomUuid(); @@ -365,7 +372,8 @@ private void getAsync() { taskName, expiryPlc, skipVals, - recovery); + recovery, + mvccSnapshot); } else { final ReaderArguments args = readerArgs; @@ -390,7 +398,8 @@ private void getAsync() { taskName, expiryPlc, skipVals, - recovery); + recovery, + mvccSnapshot); fut0.listen(createGetFutureListener()); } 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 5f976c9e16748..f56df00e9e3b3 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 @@ -975,6 +975,20 @@ public long updateCounter() { return store.updateCounter(); } + /** + * @return Current mvcc update counter value. + */ + public long mvccUpdateCounter() { + return store.mvccUpdateCounter(); + } + + /** + * @return Next mvcc update counter. + */ + public long nextMvccUpdateCounter() { + return store.nextMvccUpdateCounter(); + } + /** * @return Initial update counter. */ @@ -1258,6 +1272,15 @@ private void clearDeferredDeletes() { } } + /** + * Returns group context. + * + * @return Group context. + */ + public CacheGroupContext group() { + return grp; + } + /** * @param cacheId Cache ID. */ 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 529d96575e827..c2cd781b070e4 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 @@ -54,6 +54,9 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException; import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -82,7 +85,7 @@ * Cache lock future. */ public final class GridDhtLockFuture extends GridCacheCompoundIdentityFuture - implements GridCacheVersionedFuture, GridDhtFuture, GridCacheMappedVersion { + implements GridCacheVersionedFuture, GridDhtFuture, GridCacheMappedVersion, DhtLockFuture { /** */ private static final long serialVersionUID = 0L; @@ -264,7 +267,7 @@ public GridDhtLockFuture( if (tx != null) { while(true) { - IgniteInternalFuture fut = tx.lockFut; + IgniteInternalFuture fut = tx.lockFut; if (fut != null) { if (fut == GridDhtTxLocalAdapter.ROLLBACK_FUT) @@ -274,8 +277,8 @@ public GridDhtLockFuture( assert fut instanceof GridDhtColocatedLockFuture : fut; // Terminate this future if parent(collocated) future is terminated by rollback. - fut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture fut) { + fut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { try { fut.get(); } @@ -1307,6 +1310,10 @@ void onResult(GridDhtLockResponse res) { try { if (entry.initialValue(info.value(), info.version(), + cctx.mvccEnabled() ? ((MvccVersionAware)info).mvccVersion() : null, + cctx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccVersion() : null, + cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccTxState() : TxState.NA, + cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccTxState() : TxState.NA, info.ttl(), info.expireTime(), true, 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 42ef309eafed5..3ae3c8f726331 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 @@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.jetbrains.annotations.Nullable; @@ -74,6 +75,7 @@ public interface GridDhtPartitionTopology { public void updateTopologyVersion( GridDhtTopologyFuture exchFut, DiscoCache discoCache, + MvccCoordinator mvccCrd, long updateSeq, boolean stopping ) throws IgniteInterruptedCheckedException; @@ -421,4 +423,6 @@ public boolean update(@Nullable GridDhtPartitionExchangeId exchId, * @param updateRebalanceVer {@code True} if need check rebalance state. */ public void onExchangeDone(GridDhtPartitionsExchangeFuture fut, AffinityAssignment assignment, boolean updateRebalanceVer); + + public MvccCoordinator mvccCoordinator(); } 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 c3ff15ae5adca..1256a8bee3039 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 @@ -52,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.GridPartitionStateMap; @@ -146,6 +147,9 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology { /** */ private volatile AffinityTopologyVersion rebalancedTopVer = AffinityTopologyVersion.NONE; + /** */ + private volatile MvccCoordinator mvccCrd; + /** * @param ctx Cache shared context. * @param grp Cache group. @@ -237,6 +241,11 @@ private String mapString(GridDhtPartitionMap map) { lock.readLock().unlock(); } + /** {@inheritDoc} */ + @Override public MvccCoordinator mvccCoordinator() { + return mvccCrd; + } + /** {@inheritDoc} */ @Override public boolean holdsLock() { return lock.isWriteLockedByCurrentThread() || lock.getReadHoldCount() > 0; @@ -246,6 +255,7 @@ private String mapString(GridDhtPartitionMap map) { @Override public void updateTopologyVersion( GridDhtTopologyFuture exchFut, @NotNull DiscoCache discoCache, + MvccCoordinator mvccCrd, long updSeq, boolean stopping ) throws IgniteInterruptedCheckedException { @@ -272,6 +282,7 @@ private String mapString(GridDhtPartitionMap map) { lastTopChangeVer = exchTopVer; this.discoCache = discoCache; + this.mvccCrd = mvccCrd; } finally { lock.writeLock().unlock(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java index 866c5133b8aad..7faf0ff02d2eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.lang.IgniteProductVersion; @@ -93,11 +94,13 @@ public void validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture fu ignoringNodes.add(id); } - // Validate cache sizes. - result = validatePartitionsSizes(top, messages, ignoringNodes); + if (!MvccUtils.mvccEnabled(cctx.kernalContext())) { // TODO: Remove "if" clause in IGNITE-8149. + // Validate cache sizes. + result = validatePartitionsSizes(top, messages, ignoringNodes); - if (!result.isEmpty()) - throw new IgniteCheckedException("Partitions cache sizes are inconsistent for " + fold(topVer, result)); + if (!result.isEmpty()) + throw new IgniteCheckedException("Partitions cache sizes are inconsistent for " + fold(topVer, result)); + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsUpdateCountersMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsUpdateCountersMap.java new file mode 100644 index 0000000000000..7a345d13dfbba --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsUpdateCountersMap.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.distributed.dht; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.internal.GridDirectMap; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * Partitions update counters message. + */ +public class GridDhtPartitionsUpdateCountersMap implements Message { + /** */ + private static final long serialVersionUID = -4599730112233297219L; + + /** Map of update counters made by this tx. Mapping: partId -> updCntr. */ + @GridDirectMap(keyType = Integer.class, valueType = Long.class) + private Map updCntrs; + + /** + * + */ + public GridDhtPartitionsUpdateCountersMap() { + updCntrs = new HashMap<>(); + } + + /** + * @return Update counters. + */ + public Map updateCounters() { + return updCntrs; + } + + /** + * @param updCntrs Update counters. + */ + public void updateCounters(Map updCntrs) { + this.updCntrs = updCntrs; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeMap("updCntrs", updCntrs, MessageCollectionItemType.INT, MessageCollectionItemType.LONG)) + 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: + updCntrs = reader.readMap("updCntrs", MessageCollectionItemType.INT, MessageCollectionItemType.LONG, false); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridDhtPartitionsUpdateCountersMap.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 157; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } +} 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 f93e6a33d4e32..be1c7e282f5bb 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 @@ -28,7 +28,9 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.cluster.ClusterTopologyException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; @@ -55,8 +57,16 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTransactionalCache; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistResponse; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxRemote; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearUnlockRequest; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotWithoutTxs; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; @@ -76,6 +86,7 @@ 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.U; +import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.thread.IgniteThread; import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; @@ -83,7 +94,9 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.NOOP; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; import static org.apache.ignite.transactions.TransactionState.COMMITTING; /** @@ -164,6 +177,18 @@ protected GridDhtTransactionalCacheAdapter(GridCacheContext ctx, GridCache } }); + ctx.io().addCacheHandler(ctx.cacheId(), GridNearTxQueryEnlistRequest.class, new CI2() { + @Override public void apply(UUID nodeId, GridNearTxQueryEnlistRequest req) { + processNearTxQueryEnlistRequest(nodeId, req); + } + }); + + ctx.io().addCacheHandler(ctx.cacheId(), GridNearTxQueryEnlistResponse.class, new CI2() { + @Override public void apply(UUID nodeId, GridNearTxQueryEnlistResponse req) { + processNearEnlistResponse(nodeId, req); + } + }); + ctx.io().addCacheHandler(ctx.cacheId(), GridDhtForceKeysRequest.class, new MessageHandler() { @Override public void onMessage(ClusterNode node, GridDhtForceKeysRequest msg) { @@ -177,6 +202,41 @@ protected GridDhtTransactionalCacheAdapter(GridCacheContext ctx, GridCache processForceKeyResponse(node, msg); } }); + + ctx.io().addCacheHandler(ctx.cacheId(), GridNearTxQueryResultsEnlistRequest.class, + new CI2() { + @Override public void apply(UUID nodeId, GridNearTxQueryResultsEnlistRequest req) { + processNearTxQueryResultsEnlistRequest(nodeId, req); + } + }); + + ctx.io().addCacheHandler(ctx.cacheId(), GridNearTxQueryResultsEnlistResponse.class, + new CI2() { + @Override public void apply(UUID nodeId, GridNearTxQueryResultsEnlistResponse req) { + processNearTxEnlistResponse(nodeId, req); + } + }); + + ctx.io().addCacheHandler(ctx.cacheId(), GridDhtTxQueryEnlistRequest.class, + new CI2() { + @Override public void apply(UUID nodeId, GridDhtTxQueryEnlistRequest msg) { + processDhtTxQueryEnlistRequest(nodeId, msg, false); + } + }); + + ctx.io().addCacheHandler(ctx.cacheId(), GridDhtTxQueryFirstEnlistRequest.class, + new CI2() { + @Override public void apply(UUID nodeId, GridDhtTxQueryEnlistRequest msg) { + processDhtTxQueryEnlistRequest(nodeId, msg, true); + } + }); + + ctx.io().addCacheHandler(ctx.cacheId(), GridDhtTxQueryEnlistResponse.class, + new CI2() { + @Override public void apply(UUID nodeId, GridDhtTxQueryEnlistResponse msg) { + processDhtTxQueryEnlistResponse(nodeId, msg); + } + }); } /** {@inheritDoc} */ @@ -623,6 +683,85 @@ private void processDhtUnlockRequest(UUID nodeId, GridDhtUnlockRequest req) { near().clearLocks(nodeId, req); } + /** + * @param nodeId Node ID. + * @param req Request. + */ + private void processNearTxQueryEnlistRequest(UUID nodeId, final GridNearTxQueryEnlistRequest req) { + assert nodeId != null; + assert req != null; + + ClusterNode nearNode = ctx.discovery().node(nodeId); + + GridDhtTxLocal tx; + + try { + tx = initTxTopologyVersion(nodeId, + nearNode, + req.version(), + req.futureId(), + req.miniId(), + req.firstClientRequest(), + req.topologyVersion(), + req.threadId(), + req.txTimeout(), + req.subjectId(), + req.taskNameHash()); + } + catch (IgniteCheckedException | IgniteException ex) { + GridNearTxQueryEnlistResponse res = new GridNearTxQueryEnlistResponse(req.cacheId(), + req.futureId(), + req.miniId(), + req.version(), + ex); + + try { + ctx.io().send(nearNode, res, ctx.ioPolicy()); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send near enlist response [" + + "txId=" + req.version() + + ", node=" + nodeId + + ", res=" + res + ']', e); + } + + return; + } + + GridDhtTxQueryEnlistFuture fut = new GridDhtTxQueryEnlistFuture( + nodeId, + req.version(), + req.mvccSnapshot(), + req.threadId(), + req.futureId(), + req.miniId(), + tx, + req.cacheIds(), + req.partitions(), + req.schemaName(), + req.query(), + req.parameters(), + req.flags(), + req.pageSize(), + req.timeout(), + ctx); + + fut.listen(NearTxQueryEnlistResultHandler.instance()); + + fut.init(); + } + + /** + * @param nodeId Node ID. + * @param res Response. + */ + private void processNearEnlistResponse(UUID nodeId, final GridNearTxQueryEnlistResponse res) { + GridNearTxQueryEnlistFuture fut = (GridNearTxQueryEnlistFuture)ctx.mvcc().versionedFuture(res.version(), res.futureId()); + + if (fut != null) + fut.onResult(nodeId, res); + } + /** * @param nodeId Node ID. * @param req Request. @@ -1276,7 +1415,7 @@ private GridNearLockResponse createLockReply( CacheObject val = null; - if (ret) + if (ret) { val = e.innerGet( null, tx, @@ -1287,7 +1426,9 @@ private GridNearLockResponse createLockReply( null, tx != null ? tx.resolveTaskName() : null, null, - req.keepBinary()); + req.keepBinary(), + null); // TODO IGNITE-7371 + } assert e.lockedBy(mappedVer) || ctx.mvcc().isRemoved(e.context(), mappedVer) : "Entry does not own lock for tx [locNodeId=" + ctx.localNodeId() + @@ -1380,6 +1521,7 @@ private void sendLockReply( U.error(log, "Failed to acquire lock for request: " + req, err); try { + // TODO Async rollback // Don't send reply message to this node or if lock was cancelled or tx was rolled back asynchronously. if (!nearNode.id().equals(ctx.nodeId()) && !X.hasCause(err, GridDistributedLockCancelledException.class) && !X.hasCause(err, IgniteTxRollbackCheckedException.class)) { @@ -1576,7 +1718,7 @@ private void map(UUID nodeId, * @param nodes Nodes. * @param map Map. */ - @SuppressWarnings( {"MismatchedQueryAndUpdateOfCollection"}) + @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) private void map(GridCacheEntryEx entry, @Nullable Iterable nodes, Map> map) { @@ -1785,4 +1927,321 @@ private void obsoleteNearEntry(KeyCacheObject key) { if (nearEntry != null) nearEntry.markObsolete(ctx.versions().next()); } + + /** + * @param nodeId Node ID. + * @param req Request. + */ + private void processNearTxQueryResultsEnlistRequest(UUID nodeId, final GridNearTxQueryResultsEnlistRequest req) { + assert nodeId != null; + assert req != null; + + ClusterNode nearNode = ctx.discovery().node(nodeId); + + GridDhtTxLocal tx; + + try { + tx = initTxTopologyVersion(nodeId, + nearNode, + req.version(), + req.futureId(), + req.miniId(), + req.firstClientRequest(), + req.topologyVersion(), + req.threadId(), + req.txTimeout(), + req.subjectId(), + req.taskNameHash()); + } + catch (IgniteCheckedException | IgniteException ex) { + GridNearTxQueryResultsEnlistResponse res = new GridNearTxQueryResultsEnlistResponse(req.cacheId(), + req.futureId(), + req.miniId(), + req.version(), + ex); + + try { + ctx.io().send(nearNode, res, ctx.ioPolicy()); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send near enlist response [" + + "txId=" + req.version() + + ", node=" + nodeId + + ", res=" + res + ']', e); + } + + return; + } + + GridDhtTxQueryResultsEnlistFuture fut = new GridDhtTxQueryResultsEnlistFuture( + nodeId, + req.version(), + req.mvccSnapshot(), + req.threadId(), + req.futureId(), + req.miniId(), + tx, + req.timeout(), + ctx, + req.rows(), + req.operation()); + + fut.listen(NearTxQueryEnlistResultHandler.instance()); + + fut.init(); + } + + /** + * @param nodeId Near node id. + * @param nearNode Near node. + * @param nearLockVer Near lock version. + * @param nearFutId Near future id. + * @param nearMiniId Near mini-future id. + * @param firstClientReq First client request flag. + * @param topVer Topology version. + * @param nearThreadId Near node thread id. + * @param timeout Timeout. + * @param txSubjectId Transaction subject id. + * @param txTaskNameHash Transaction task name hash. + * @return Transaction. + */ + public GridDhtTxLocal initTxTopologyVersion(UUID nodeId, + ClusterNode nearNode, + GridCacheVersion nearLockVer, + IgniteUuid nearFutId, + int nearMiniId, + boolean firstClientReq, + AffinityTopologyVersion topVer, + long nearThreadId, + long timeout, + UUID txSubjectId, + int txTaskNameHash) throws IgniteException, IgniteCheckedException { + + assert ctx.affinityNode(); + + if (txLockMsgLog.isDebugEnabled()) { + txLockMsgLog.debug("Received near enlist request [txId=" + nearLockVer + + ", node=" + nodeId + ']'); + } + + if (nearNode == null) { + U.warn(txLockMsgLog, "Received near enlist request from unknown node (will ignore) [txId=" + nearLockVer + + ", node=" + nodeId + ']'); + + return null; + } + + GridDhtTxLocal tx = null; + + GridCacheVersion dhtVer = ctx.tm().mappedVersion(nearLockVer); + + if (dhtVer != null) + tx = ctx.tm().tx(dhtVer); + + GridDhtPartitionTopology top = null; + + if (tx == null) { + if (firstClientReq) { + assert nearNode.isClient(); + + top = topology(); + + top.readLock(); + + GridDhtTopologyFuture topFut = top.topologyVersionFuture(); + + if (!topFut.isDone() || !topFut.topologyVersion().equals(topVer)) { + // TODO IGNITE-7164 Wait for topology change, remap client TX in case affinity was changed. + top.readUnlock(); + + throw new ClusterTopologyException("Topology was changed. Please retry on stable topology."); + } + } + + try { + tx = new GridDhtTxLocal( + ctx.shared(), + topVer, + nearNode.id(), + nearLockVer, + nearFutId, + nearMiniId, + nearThreadId, + false, + false, + ctx.systemTx(), + false, + ctx.ioPolicy(), + PESSIMISTIC, + REPEATABLE_READ, + timeout, + false, + false, + false, + -1, + null, + txSubjectId, + txTaskNameHash); + + // if (req.syncCommit()) + tx.syncMode(FULL_SYNC); + + tx = ctx.tm().onCreated(null, tx); + + if (tx == null || !tx.init()) { + String msg = "Failed to acquire lock (transaction has been completed): " + + nearLockVer; + + U.warn(log, msg); + + try { + if (tx != null) + tx.rollbackDhtLocal(); + } + catch (IgniteCheckedException ex) { + U.error(log, "Failed to rollback the transaction: " + tx, ex); + } + + throw new IgniteCheckedException(msg); + } + + tx.topologyVersion(topVer); + } + finally { + if (top != null) + top.readUnlock(); + } + } + + ctx.tm().txContext(tx); + + return tx; + } + + /** + * @param nodeId Node ID. + * @param res Response. + */ + private void processNearTxEnlistResponse(UUID nodeId, final GridNearTxQueryResultsEnlistResponse res) { + GridNearTxQueryResultsEnlistFuture fut = (GridNearTxQueryResultsEnlistFuture) + ctx.mvcc().versionedFuture(res.version(), res.futureId()); + + if (fut != null) + fut.onResult(nodeId, res); + } + + /** + * @param primary Primary node. + * @param req Request. + * @param e Error. + */ + private void onError(UUID primary, GridDhtTxQueryEnlistRequest req, Throwable e) { + GridDhtTxQueryEnlistResponse res = new GridDhtTxQueryEnlistResponse(ctx.cacheId(), + req.dhtFutureId(), + req.batchId(), + e); + + try { + ctx.io().send(primary, res, ctx.ioPolicy()); + } + catch (IgniteCheckedException ioEx) { + U.error(log, "Failed to send DHT enlist reply to primary node [node: " + primary + ", req=" + req + + ']', ioEx); + } + } + + /** + * @param primary Primary node. + * @param req Message. + * @param first Flag if this is a first request in current operation. + */ + private void processDhtTxQueryEnlistRequest(UUID primary, GridDhtTxQueryEnlistRequest req, boolean first) { + try { + assert req.version() != null && req.op() != null; + + GridDhtTxRemote tx = ctx.tm().tx(req.version()); + + if (tx == null) { + if (!first) + throw new IgniteCheckedException("Can not find a transaction for version [version=" + + req.version() + ']'); + + GridDhtTxQueryFirstEnlistRequest req0 = (GridDhtTxQueryFirstEnlistRequest)req; + + tx = new GridDhtTxRemote(ctx.shared(), + req0.nearNodeId(), + req0.dhtFutureId(), + primary, + req0.nearXidVersion(), + req0.topologyVersion(), + req0.version(), + null, + ctx.systemTx(), + ctx.ioPolicy(), + PESSIMISTIC, + REPEATABLE_READ, + false, + req0.timeout(), + -1, + req0.subjectId(), + req0.taskNameHash(), + false); + + tx.mvccSnapshot(new MvccSnapshotWithoutTxs(req0.coordinatorVersion(), req0.counter(), + MVCC_OP_COUNTER_NA, req0.cleanupVersion())); + + tx = ctx.tm().onCreated(null, tx); + + if (tx == null || !ctx.tm().onStarted(tx)) { + throw new IgniteTxRollbackCheckedException("Failed to update backup " + + "(transaction has been completed): " + req0.version()); + } + } + + assert tx != null; + + MvccSnapshot s0 = tx.mvccSnapshot(); + + MvccSnapshot snapshot = new MvccSnapshotWithoutTxs(s0.coordinatorVersion(), s0.counter(), + req.operationCounter(), s0.cleanupVersion()); + + tx.mvccEnlistBatch(ctx, req.op(), req.keys(), req.values(), snapshot, req.updateCounters()); + + GridDhtTxQueryEnlistResponse res = new GridDhtTxQueryEnlistResponse(req.cacheId(), + req.dhtFutureId(), + req.batchId(), + null); + + try { + ctx.io().send(primary, res, ctx.ioPolicy()); + } + catch (IgniteCheckedException ioEx) { + U.error(log, "Failed to send DHT enlist reply to primary node [node: " + primary + ", req=" + + req + ']', ioEx); + } + } + catch (IgniteCheckedException e) { + onError(primary, req, e); + } + } + + /** + * @param backup Backup node. + * @param res Response message. + */ + private void processDhtTxQueryEnlistResponse(UUID backup, GridDhtTxQueryEnlistResponse res) { + GridDhtTxAbstractEnlistFuture fut = (GridDhtTxAbstractEnlistFuture) + ctx.mvcc().future(res.futureId()); + + if (fut == null) { + U.warn(log, "Received dht enlist response for unknown future [futId=" + res.futureId() + + ", batchId=" + res.batchId() + + ", node=" + backup + ']'); + + return; + } + + fut.onResult(backup, res); + } + } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java new file mode 100644 index 0000000000000..bb11df5590c62 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java @@ -0,0 +1,1139 @@ +/* + * 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.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; +import org.apache.ignite.internal.processors.cache.GridCacheFutureAdapter; +import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxAbstractEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxSelectForUpdateFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.F; +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.IgniteInClosure; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Abstract future processing transaction enlisting and locking + * of entries produced with DML and SELECT FOR UPDATE queries. + */ +public abstract class GridDhtTxAbstractEnlistFuture extends GridCacheFutureAdapter + implements DhtLockFuture { + /** Done field updater. */ + private static final AtomicIntegerFieldUpdater DONE_UPD = + AtomicIntegerFieldUpdater.newUpdater(GridDhtTxAbstractEnlistFuture.class, "done"); + + /** SkipCntr field updater. */ + private static final AtomicIntegerFieldUpdater SKIP_UPD = + AtomicIntegerFieldUpdater.newUpdater(GridDhtTxAbstractEnlistFuture.class, "skipCntr"); + + /** Marker object. */ + private static final Object FINISHED = new Object(); + + /** */ + private static final int BATCH_SIZE = 1024; + + /** In-flight batches per node limit. */ + private static final int BATCHES_PER_NODE = 5; + + /** */ + private static final int FIRST_BATCH_ID = 0; + + /** Future ID. */ + protected final IgniteUuid futId; + + /** Cache registry. */ + @GridToStringExclude + protected final GridCacheContext cctx; + + /** Logger. */ + @GridToStringExclude + protected final IgniteLogger log; + + /** Thread. */ + protected final long threadId; + + /** Future ID. */ + protected final IgniteUuid nearFutId; + + /** Future ID. */ + protected final int nearMiniId; + + /** Partitions. */ + protected final int[] parts; + + /** Transaction. */ + protected final GridDhtTxLocalAdapter tx; + + /** Lock version. */ + protected final GridCacheVersion lockVer; + + /** */ + protected final MvccSnapshot mvccSnapshot; + + /** Processed entries count. */ + protected long cnt; + + /** Near node ID. */ + protected final UUID nearNodeId; + + /** Near lock version. */ + protected final GridCacheVersion nearLockVer; + + /** Timeout object. */ + @GridToStringExclude + protected LockTimeoutObject timeoutObj; + + /** Lock timeout. */ + protected final long timeout; + + /** Query iterator */ + private UpdateSourceIterator it; + + /** Row extracted from iterator but not yet used. */ + private Object peek; + + /** */ + @SuppressWarnings({"FieldCanBeLocal"}) + @GridToStringExclude + private volatile int skipCntr; + + /** */ + @SuppressWarnings("unused") + @GridToStringExclude + private volatile int done; + + /** */ + @GridToStringExclude + private int batchIdCntr; + + /** Batches for sending to remote nodes. */ + private Map batches; + + /** Batches already sent to remotes, but their acks are not received yet. */ + private ConcurrentMap> pending; + + /** */ + private WALPointer walPtr; + + /** Do not send DHT requests to near node. */ + protected boolean skipNearNodeUpdates; + + /** There are keys belonging to backup partitions on near node. */ + protected boolean hasNearNodeUpdates; + + /** Moving partitions. */ + private Map movingParts; + + /** Update counters to be sent to the near node in case it is a backup node also. */ + protected GridLongList nearUpdCntrs; + + /** + * @param nearNodeId Near node ID. + * @param nearLockVer Near lock version. + * @param mvccSnapshot Mvcc snapshot. + * @param threadId Thread ID. + * @param nearFutId Near future id. + * @param nearMiniId Near mini future id. + * @param parts Partitions. + * @param tx Transaction. + * @param timeout Lock acquisition timeout. + * @param cctx Cache context. + */ + protected GridDhtTxAbstractEnlistFuture(UUID nearNodeId, + GridCacheVersion nearLockVer, + MvccSnapshot mvccSnapshot, + long threadId, + IgniteUuid nearFutId, + int nearMiniId, + @Nullable int[] parts, + GridDhtTxLocalAdapter tx, + long timeout, + GridCacheContext cctx) { + assert tx != null; + assert timeout >= 0; + assert nearNodeId != null; + assert nearLockVer != null; + assert threadId == tx.threadId(); + + this.threadId = threadId; + this.cctx = cctx; + this.nearNodeId = nearNodeId; + this.nearLockVer = nearLockVer; + this.nearFutId = nearFutId; + this.nearMiniId = nearMiniId; + this.mvccSnapshot = mvccSnapshot; + this.timeout = timeout; + this.tx = tx; + this.parts = parts; + + lockVer = tx.xidVersion(); + + futId = IgniteUuid.randomUuid(); + + log = cctx.logger(GridDhtTxAbstractEnlistFuture.class); + } + + /** + * @return iterator. + * @throws IgniteCheckedException If failed. + */ + protected abstract UpdateSourceIterator createIterator() throws IgniteCheckedException; + + /** + * + */ + public void init() { + if (timeout < 0) { + // Time is out. + onDone(timeoutException()); + + return; + } + else if (timeout > 0) + timeoutObj = new LockTimeoutObject(); + + while(true) { + IgniteInternalFuture fut = tx.lockFut; + + if (fut == GridDhtTxLocalAdapter.ROLLBACK_FUT) { + onDone(tx.timedOut() ? tx.timeoutException() : tx.rollbackException()); + + return; + } + else if (fut != null) { + // Wait for previous future. + assert fut instanceof GridNearTxAbstractEnlistFuture + || fut instanceof GridDhtTxAbstractEnlistFuture + || fut instanceof CompoundLockFuture + || fut instanceof GridNearTxSelectForUpdateFuture : fut; + + // Terminate this future if parent future is terminated by rollback. + if (!fut.isDone()) { + fut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + if (fut.error() != null) + onDone(fut.error()); + } + }); + } + else if (fut.error() != null) + onDone(fut.error()); + + break; + } + else if (tx.updateLockFuture(null, this)) + break; + } + + boolean added = cctx.mvcc().addFuture(this, futId); + + assert added; + + if (isDone()) { + cctx.mvcc().removeFuture(futId); + + return; + } + + if (timeoutObj != null) + cctx.time().addTimeoutObject(timeoutObj); + + try { + checkPartitions(parts); + + UpdateSourceIterator it = createIterator(); + + if (!it.hasNext()) { + U.close(it, log); + + onDone(0L); + + return; + } + + tx.addActiveCache(cctx, false); + + this.it = it; + } + catch (Throwable e) { + onDone(e); + + if (e instanceof Error) + throw (Error)e; + + return; + } + + continueLoop(false); + } + + /** + * Clears lock future. + */ + protected void clearLockFuture() { + tx.clearLockFuture(this); + } + + /** + * Iterates over iterator, applies changes locally and sends it on backups. + * + * @param ignoreCntr {@code True} if need to ignore skip counter. + */ + private void continueLoop(boolean ignoreCntr) { + if (isDone() || (!ignoreCntr && (SKIP_UPD.getAndIncrement(this) != 0))) + return; + + GridDhtCacheAdapter cache = cctx.dhtCache(); + EnlistOperation op = it.operation(); + AffinityTopologyVersion topVer = tx.topologyVersionSnapshot(); + + try { + while (true) { + while (hasNext0()) { + Object cur = next0(); + + KeyCacheObject key = cctx.toCacheKeyObject(op.isDeleteOrLock() ? cur : ((IgniteBiTuple)cur).getKey()); + + if (!ensureFreeSlot(key)) { + // Can't advance further at the moment. + peek = cur; + + it.beforeDetach(); + + break; + } + + GridDhtCacheEntry entry = cache.entryExx(key); + + if (log.isDebugEnabled()) + log.debug("Adding entry: " + entry); + + assert !entry.detached(); + + CacheObject val = op.isDeleteOrLock() ? null : cctx.toCacheObject(((IgniteBiTuple)cur).getValue()); + + tx.markQueryEnlisted(mvccSnapshot); + + GridCacheUpdateTxResult res; + + while (true) { + cctx.shared().database().checkpointReadLock(); + + try { + switch (op) { + case DELETE: + res = entry.mvccRemove( + tx, + cctx.localNodeId(), + topVer, + null, + mvccSnapshot, + isMoving(key.partition())); + + break; + + case INSERT: + case UPSERT: + case UPDATE: + res = entry.mvccSet( + tx, + cctx.localNodeId(), + val, + 0, + topVer, + null, + mvccSnapshot, + op.cacheOperation(), + isMoving(key.partition()), + op.noCreate()); + + break; + + case LOCK: + res = entry.mvccLock( + tx, + mvccSnapshot); + + break; + + default: + throw new IgniteSQLException("Cannot acquire lock for operation [op= " + op + "]" + + "Operation is unsupported at the moment ", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + } + + break; + } + catch (GridCacheEntryRemovedException ignored) { + entry = cache.entryExx(entry.key(), topVer); + } + finally { + cctx.shared().database().checkpointReadUnlock(); + } + } + + IgniteInternalFuture updateFut = res.updateFuture(); + + if (updateFut != null) { + if (updateFut.isDone()) + res = updateFut.get(); + else { + CacheObject val0 = val; + GridDhtCacheEntry entry0 = entry; + + it.beforeDetach(); + + updateFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + processEntry(entry0, op, fut.get(), val0); + + continueLoop(true); + } + catch (Throwable e) { + onDone(e); + } + } + }); + + // Can't move further. Exit loop without decrementing the counter. + return; + } + } + + processEntry(entry, op, res, val); + } + + if (!hasNext0()) { + if (walPtr != null && !cctx.tm().logTxRecords()) { + cctx.shared().wal().flush(walPtr, true); + + walPtr = null; // Avoid additional flushing. + } + + if (!F.isEmpty(batches)) { + // Flush incomplete batches. + // Need to skip batches for nodes where first request (contains tx info) is still in-flight. + // Otherwise, the regular enlist request (without tx info) may beat it to the primary node. + Iterator> it = batches.entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry e = it.next(); + + ConcurrentMap pending0 = + pending == null ? null : pending.get(e.getKey()); + + if (pending0 == null || !pending0.containsKey(FIRST_BATCH_ID)) { + it.remove(); + + sendBatch(e.getValue()); + } + } + } + + if (noPendingRequests()) { + onDone(cnt); + + return; + } + } + + if (SKIP_UPD.decrementAndGet(this) == 0) + break; + + skipCntr = 1; + } + } + catch (Throwable e) { + onDone(e); + + if (e instanceof Error) + throw (Error)e; + } + } + + /** */ + private Object next0() { + if (!hasNext0()) + throw new NoSuchElementException(); + + Object cur; + + if ((cur = peek) != null) + peek = null; + else + cur = it.next(); + + return cur; + } + + /** */ + private boolean hasNext0() { + if (peek == null && !it.hasNext()) + peek = FINISHED; + + return peek != FINISHED; + } + + /** + * @return {@code True} if in-flight batches map is empty. + */ + private boolean noPendingRequests() { + if (F.isEmpty(pending)) + return true; + + for (ConcurrentMap e : pending.values()) { + if (!e.isEmpty()) + return false; + } + + return true; + } + + /** + * @param entry Cache entry. + * @param op Operation. + * @param updRes Update result. + * @param val New value. + * @throws IgniteCheckedException If failed. + */ + private void processEntry(GridDhtCacheEntry entry, EnlistOperation op, + GridCacheUpdateTxResult updRes, CacheObject val) throws IgniteCheckedException { + checkCompleted(); + + assert updRes != null && updRes.updateFuture() == null; + + WALPointer ptr0 = updRes.loggedPointer(); + + if (ptr0 != null) + walPtr = ptr0; + + if (!updRes.success()) + return; + + cnt++; + + if (op != EnlistOperation.LOCK) + addToBatch(entry.key(), val, updRes.mvccHistory(), updRes.updateCounter(), entry.context().cacheId()); + } + + /** + * Adds row to batch. + * IMPORTANT: This method should be called from the critical section in {@link this.sendNextBatches()} + * + * @param key Key. + * @param val Value. + * @param hist History rows. + * @param updCntr Update counter. + */ + private void addToBatch(KeyCacheObject key, CacheObject val, List hist, long updCntr, + int cacheId) throws IgniteCheckedException { + List backups = backupNodes(key); + + int part = cctx.affinity().partition(key); + + tx.addPartitionCountersMapping(cacheId, part); + + if (F.isEmpty(backups)) + return; + + CacheEntryInfoCollection hist0 = null; + + for (ClusterNode node : backups) { + assert !node.isLocal(); + + boolean moving = isMoving(node, part); + + if (skipNearNodeUpdates && node.id().equals(nearNodeId) && !moving) { + updateMappings(node); + + if (newRemoteTx(node)) + tx.addLockTransactionNode(node); + + hasNearNodeUpdates = true; + + if (nearUpdCntrs == null) + nearUpdCntrs = new GridLongList(); + + nearUpdCntrs.add(updCntr); + + continue; + } + + Batch batch = null; + + if (batches == null) + batches = new HashMap<>(); + else + batch = batches.get(node.id()); + + if (batch == null) + batches.put(node.id(), batch = new Batch(node)); + + if (moving && hist0 == null) { + assert !F.isEmpty(hist); + + hist0 = fetchHistoryInfo(key, hist); + } + + batch.add(key, moving ? hist0 : val, updCntr); + + if (batch.size() == BATCH_SIZE) { + assert batches != null; + + batches.remove(node.id()); + + sendBatch(batch); + } + } + } + + /** + * + * @param key Key. + * @param hist History rows. + * @return History entries. + * @throws IgniteCheckedException, if failed. + */ + private CacheEntryInfoCollection fetchHistoryInfo(KeyCacheObject key, List hist) + throws IgniteCheckedException { + List res = new ArrayList<>(); + + for (int i = 0; i < hist.size(); i++) { + MvccLinkAwareSearchRow row0 = hist.get(i); + + MvccDataRow row = new MvccDataRow(cctx.group(), + row0.hash(), + row0.link(), + key.partition(), + CacheDataRowAdapter.RowData.NO_KEY, + row0.mvccCoordinatorVersion(), + row0.mvccCounter(), + row0.mvccOperationCounter()); + + GridCacheMvccEntryInfo entry = new GridCacheMvccEntryInfo(); + + entry.version(row.version()); + entry.mvccVersion(row); + entry.newMvccVersion(row); + entry.value(row.value()); + entry.expireTime(row.expireTime()); + + if (MvccUtils.compare(mvccSnapshot, row.mvccCoordinatorVersion(), row.mvccCounter()) != 0) { + entry.mvccTxState(row.mvccTxState() != TxState.NA ? row.mvccTxState() : + MvccUtils.state(cctx, row.mvccCoordinatorVersion(), row.mvccCounter(), row.mvccOperationCounter())); + } + + if (MvccUtils.compare(mvccSnapshot, row.newMvccCoordinatorVersion(), row.newMvccCounter()) != 0) { + entry.newMvccTxState(row.newMvccTxState() != TxState.NA ? row.newMvccTxState() : + MvccUtils.state(cctx, row.newMvccCoordinatorVersion(), row.newMvccCounter(), + row.newMvccOperationCounter())); + } + + res.add(entry); + } + + return new CacheEntryInfoCollection(res); + } + + /** */ + private boolean newRemoteTx(ClusterNode node) { + Set nodes = tx.lockTransactionNodes(); + + return nodes == null || !nodes.contains(node); + } + + /** + * Checks if there free space in batches or free slot in in-flight batches is available for the given key. + * + * @param key Key. + * @return {@code True} if there is possible to add this key to batch or send ready batch. + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + private boolean ensureFreeSlot(KeyCacheObject key) { + if (F.isEmpty(batches) || F.isEmpty(pending)) + return true; + + // Check possibility of adding to batch and sending. + for (ClusterNode node : backupNodes(key)) { + if (skipNearNodeUpdates && node.id().equals(nearNodeId) && !isMoving(node, key.partition())) + continue; + + Batch batch = batches.get(node.id()); + + // We can add key if batch is not full. + if (batch == null || batch.size() < BATCH_SIZE - 1) + continue; + + ConcurrentMap pending0 = pending.get(node.id()); + + assert pending0 == null || pending0.size() <= BATCHES_PER_NODE; + + if (pending0 != null && (pending0.containsKey(FIRST_BATCH_ID) || pending0.size() == BATCHES_PER_NODE)) + return false; + } + + return true; + } + + /** + * Send batch request to remote data node. + * + * @param batch Batch. + */ + private void sendBatch(Batch batch) throws IgniteCheckedException { + assert batch != null && !batch.node().isLocal(); + + ClusterNode node = batch.node(); + + updateMappings(node); + + GridDhtTxQueryEnlistRequest req; + + if (newRemoteTx(node)) { + tx.addLockTransactionNode(node); + + // If this is a first request to this node, send full info. + req = new GridDhtTxQueryFirstEnlistRequest(cctx.cacheId(), + futId, + cctx.localNodeId(), + tx.topologyVersionSnapshot(), + lockVer, + mvccSnapshot, + tx.remainingTime(), + tx.taskNameHash(), + nearNodeId, + nearLockVer, + it.operation(), + FIRST_BATCH_ID, + batch.keys(), + batch.values(), + batch.updateCounters()); + } + else { + // Send only keys, values, LockVersion and batchId if this is not a first request to this backup. + req = new GridDhtTxQueryEnlistRequest(cctx.cacheId(), + futId, + lockVer, + it.operation(), + ++batchIdCntr, + mvccSnapshot.operationCounter(), + batch.keys(), + batch.values(), + batch.updateCounters()); + } + + ConcurrentMap pending0 = null; + + if (pending == null) + pending = new ConcurrentHashMap<>(); + else + pending0 = pending.get(node.id()); + + if (pending0 == null) + pending.put(node.id(), pending0 = new ConcurrentHashMap<>()); + + Batch prev = pending0.put(req.batchId(), batch); + + assert prev == null; + + cctx.io().send(node, req, cctx.ioPolicy()); + } + + /** */ + private synchronized void updateMappings(ClusterNode node) throws IgniteCheckedException { + checkCompleted(); + + Map m = tx.dhtMap; + + GridDistributedTxMapping mapping = m.get(node.id()); + + if (mapping == null) + m.put(node.id(), mapping = new GridDistributedTxMapping(node)); + + mapping.markQueryUpdate(); + } + + /** + * @param key Key. + * @return Backup nodes for the given key. + */ + @NotNull private List backupNodes(KeyCacheObject key) { + List dhtNodes = cctx.affinity().nodesByKey(key, tx.topologyVersion()); + + assert !dhtNodes.isEmpty() && dhtNodes.get(0).id().equals(cctx.localNodeId()) : + "localNode = " + cctx.localNodeId() + ", dhtNodes = " + dhtNodes; + + if (dhtNodes.size() == 1) + return Collections.emptyList(); + + return dhtNodes.subList(1, dhtNodes.size()); + } + + /** + * Checks whether all the necessary partitions are in {@link GridDhtPartitionState#OWNING} state. + * + * @param parts Partitions. + * @throws ClusterTopologyCheckedException If failed. + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + private void checkPartitions(@Nullable int[] parts) throws ClusterTopologyCheckedException { + if (cctx.isLocal() || !cctx.rebalanceEnabled()) + return; + + if (parts == null) + parts = U.toIntArray( + cctx.affinity() + .primaryPartitions(cctx.localNodeId(), tx.topologyVersionSnapshot())); + + GridDhtPartitionTopology top = cctx.topology(); + + try { + top.readLock(); + + for (int i = 0; i < parts.length; i++) { + GridDhtLocalPartition p = top.localPartition(parts[i]); + + if (p == null || p.state() != GridDhtPartitionState.OWNING) + throw new ClusterTopologyCheckedException("Cannot run update query. " + + "Node must own all the necessary partitions."); // TODO IGNITE-7185 Send retry instead. + } + } + finally { + top.readUnlock(); + } + } + + /** + * @param part Partition. + * @return {@code true} if the given partition is rebalancing to any backup node. + */ + private boolean isMoving(int part) { + if (movingParts == null) + movingParts = new HashMap<>(); + + Boolean res = movingParts.get(part); + + if (res != null) + return res; + + List dhtNodes = cctx.affinity().nodesByPartition(part, tx.topologyVersion()); + + for (int i = 1; i < dhtNodes.size(); i++) { + ClusterNode node = dhtNodes.get(i); + if (isMoving(node, part)) { + movingParts.put(part, Boolean.TRUE); + + return true; + } + } + + movingParts.put(part, Boolean.FALSE); + + return false; + } + + /** + * @param node Cluster node. + * @param part Partition. + * @return {@code true} if the given partition is rebalancing to the given node. + */ + private boolean isMoving(ClusterNode node, int part) { + GridDhtPartitionState partState = cctx.topology().partitionState(node.id(), part); + + return partState != GridDhtPartitionState.OWNING && partState != GridDhtPartitionState.EVICTED; + } + + /** */ + private void checkCompleted() throws IgniteCheckedException { + if (isDone()) + throw new IgniteCheckedException("Future is done."); + } + + /** + * Callback on backup response. + * + * @param nodeId Backup node. + * @param res Response. + */ + public void onResult(UUID nodeId, GridDhtTxQueryEnlistResponse res) { + if (res.error() != null) { + onDone(new IgniteCheckedException("Failed to update backup node: [localNodeId=" + cctx.localNodeId() + + ", remoteNodeId=" + nodeId + ']', res.error())); + + return; + } + + assert pending != null; + + ConcurrentMap pending0 = pending.get(nodeId); + + assert pending0 != null; + + Batch rmv = pending0.remove(res.batchId()); + + assert rmv != null; + + continueLoop(false); + } + + /** {@inheritDoc} */ + @Override public boolean trackable() { + return true; + } + + /** {@inheritDoc} */ + @Override public void markNotTrackable() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public IgniteUuid futureId() { + return futId; + } + + /** {@inheritDoc} */ + @Override public boolean onNodeLeft(UUID nodeId) { + boolean backupLeft = false; + + Set nodes = tx.lockTransactionNodes(); + + if (!F.isEmpty(nodes)) { + for (ClusterNode node : nodes) { + if (node.id().equals(nodeId)) { + backupLeft = true; + + break; + } + } + } + + return (backupLeft || nearNodeId.equals(nodeId)) && onDone( + new ClusterTopologyCheckedException((backupLeft ? "Backup" : "Requesting") + + " node left the grid [nodeId=" + nodeId + ']')); + } + + /** {@inheritDoc} */ + @Override public boolean onDone(@Nullable Long res, @Nullable Throwable err) { + assert res != null || err != null; + + if (!DONE_UPD.compareAndSet(this, 0, 1)) + return false; + + if (err == null) + clearLockFuture(); + + // To prevent new remote transactions creation + // after future is cancelled by rollback. + synchronized (this) { + boolean done = super.onDone(res, err); + + assert done; + + if (log.isDebugEnabled()) + log.debug("Completing future: " + this); + + // Clean up. + cctx.mvcc().removeFuture(futId); + + if (timeoutObj != null) + cctx.time().removeTimeoutObject(timeoutObj); + + U.close(it, log); + + return true; + } + } + + /** {@inheritDoc} */ + @Override public void onError(Throwable error) { + onDone(error); + } + + /** + * @return Timeout exception. + */ + @NotNull protected IgniteTxTimeoutCheckedException timeoutException() { + return new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " + + "transaction [timeout=" + timeout + ", tx=" + tx + ']'); + } + + /** + * A batch of rows + */ + private static class Batch { + /** Node ID. */ + @GridToStringExclude + private final ClusterNode node; + + /** */ + private List keys; + + /** + * Values collection. + * Items can be either {@link CacheObject} or preload entries collection {@link CacheEntryInfoCollection}. + */ + private List vals; + + /** Update counters. */ + private GridLongList updCntrs; + + /** + * @param node Cluster node. + */ + private Batch(ClusterNode node) { + this.node = node; + } + + /** + * @return Node. + */ + public ClusterNode node() { + return node; + } + + /** + * Adds a row to batch. + * + * @param key Key. + * @param val Value or preload entries collection. + */ + public void add(KeyCacheObject key, Message val, long updCntr) { + assert val == null || val instanceof CacheObject || val instanceof CacheEntryInfoCollection; + assert updCntr > 0; + + if (keys == null) + keys = new ArrayList<>(); + + keys.add(key); + + if (val != null) { + if (vals == null) + vals = new ArrayList<>(); + + vals.add(val); + } + + if (updCntrs == null) + updCntrs = new GridLongList(); + + updCntrs.add(updCntr); + + assert (vals == null) || keys.size() == vals.size(); + } + + /** + * @return number of rows. + */ + public int size() { + return keys == null ? 0 : keys.size(); + } + + /** + * @return Collection of row keys. + */ + public List keys() { + return keys; + } + + /** + * @return Collection of row values. + */ + public List values() { + return vals; + } + + /** + * @return Update counters. + */ + public GridLongList updateCounters() { + return updCntrs; + } + } + + /** + * Lock request timeout object. + */ + protected class LockTimeoutObject extends GridTimeoutObjectAdapter { + /** + * Default constructor. + */ + LockTimeoutObject() { + super(timeout); + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (log.isDebugEnabled()) + log.debug("Timed out waiting for lock response: " + this); + + onDone(timeoutException()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(LockTimeoutObject.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java index 2f36053b9d838..c0a38452b279d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java @@ -26,19 +26,23 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.IgniteDiagnosticAware; import org.apache.ignite.internal.IgniteDiagnosticPrepareContext; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheCompoundIdentityFuture; import org.apache.ignite.internal.processors.cache.GridCacheFuture; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; +import org.apache.ignite.internal.processors.cache.mvcc.MvccFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -284,6 +288,8 @@ private void onComplete() { public void finish(boolean commit) { boolean sync; + assert !tx.queryEnlisted() || tx.mvccSnapshot() != null; + if (!F.isEmpty(dhtMap) || !F.isEmpty(nearMap)) sync = finish(commit, dhtMap, nearMap); else if (!commit && !F.isEmpty(tx.lockTransactionNodes())) @@ -292,6 +298,22 @@ else if (!commit && !F.isEmpty(tx.lockTransactionNodes())) // No backup or near nodes to send commit message to (just complete then). sync = false; + GridLongList waitTxs = tx.mvccWaitTransactions(); + + if (waitTxs != null) { + MvccSnapshot snapshot = tx.mvccSnapshot(); + + assert snapshot != null; + + MvccCoordinator crd = cctx.coordinators().currentCoordinator(); + + if (crd != null && crd.coordinatorVersion() == snapshot.coordinatorVersion()) { + add((IgniteInternalFuture)cctx.coordinators().waitTxsFuture(crd.nodeId(), waitTxs)); + + sync = true; + } + } + markInitialized(); if (!sync) @@ -310,7 +332,7 @@ private boolean rollbackLockTransactions(Collection nodes) { boolean sync = tx.syncMode() == FULL_SYNC; - if (tx.explicitLock()) + if (tx.explicitLock() || tx.queryEnlisted()) sync = true; boolean res = false; @@ -348,7 +370,8 @@ private boolean rollbackLockTransactions(Collection nodes) { tx.taskNameHash(), tx.activeCachesDeploymentEnabled(), false, - false); + false, + tx.mvccSnapshot()); try { cctx.io().send(n, req, tx.ioPolicy()); @@ -396,15 +419,23 @@ private boolean finish(boolean commit, if (tx.onePhaseCommit()) return false; + assert !commit || !tx.txState().mvccEnabled(cctx) || tx.mvccSnapshot() != null || F.isEmpty(tx.writeEntries()); + boolean sync = tx.syncMode() == FULL_SYNC; - if (tx.explicitLock()) + if (tx.explicitLock() || tx.queryEnlisted()) sync = true; boolean res = false; int miniId = 0; + // Do not need process active transactions on backups. + MvccSnapshot mvccSnapshot = tx.mvccSnapshot(); + + if (mvccSnapshot != null) + mvccSnapshot = mvccSnapshot.withoutActiveTransactions(); + // Create mini futures. for (GridDistributedTxMapping dhtMapping : dhtMap.values()) { ClusterNode n = dhtMapping.primary(); @@ -413,7 +444,7 @@ private boolean finish(boolean commit, GridDistributedTxMapping nearMapping = nearMap.get(n.id()); - if (dhtMapping.empty() && nearMapping != null && nearMapping.empty()) + if (!dhtMapping.queryUpdate() && dhtMapping.empty() && nearMapping != null && nearMapping.empty()) // Nothing to send. continue; @@ -426,6 +457,11 @@ private boolean finish(boolean commit, for (IgniteTxEntry e : dhtMapping.entries()) updCntrs.add(e.updateCounter()); + Map updCntrsMap = null; + + if (dhtMapping.queryUpdate() && commit) + updCntrsMap = tx.updateCountersForNode(n); + GridDhtTxFinishRequest req = new GridDhtTxFinishRequest( tx.nearNodeId(), futId, @@ -451,7 +487,9 @@ private boolean finish(boolean commit, tx.activeCachesDeploymentEnabled(), updCntrs, false, - false); + false, + mvccSnapshot, + updCntrsMap); req.writeVersion(tx.writeVersion() != null ? tx.writeVersion() : tx.xidVersion()); @@ -520,7 +558,8 @@ private boolean finish(boolean commit, tx.taskNameHash(), tx.activeCachesDeploymentEnabled(), false, - false); + false, + mvccSnapshot); req.writeVersion(tx.writeVersion()); @@ -565,22 +604,35 @@ private boolean finish(boolean commit, if (!isDone()) { for (IgniteInternalFuture fut : futures()) { if (!fut.isDone()) { - MiniFuture f = (MiniFuture)fut; - - if (!f.node().isLocal()) { - GridCacheVersion dhtVer = tx.xidVersion(); - GridCacheVersion nearVer = tx.nearXidVersion(); + if (MiniFuture.class.isInstance(fut)) { + MiniFuture f = (MiniFuture)fut; + + if (!f.node().isLocal()) { + GridCacheVersion dhtVer = tx.xidVersion(); + GridCacheVersion nearVer = tx.nearXidVersion(); + + ctx.remoteTxInfo(f.node().id(), dhtVer, nearVer, "GridDhtTxFinishFuture " + + "waiting for response [node=" + f.node().id() + + ", topVer=" + tx.topologyVersion() + + ", dhtVer=" + dhtVer + + ", nearVer=" + nearVer + + ", futId=" + futId + + ", miniId=" + f.futId + + ", tx=" + tx + ']'); + + return; + } + } + else if (fut instanceof MvccFuture) { + MvccFuture f = (MvccFuture)fut; - ctx.remoteTxInfo(f.node().id(), dhtVer, nearVer, "GridDhtTxFinishFuture " + - "waiting for response [node=" + f.node().id() + - ", topVer=" + tx.topologyVersion() + - ", dhtVer=" + dhtVer + - ", nearVer=" + nearVer + - ", futId=" + futId + - ", miniId=" + f.futId + - ", tx=" + tx + ']'); + if (!cctx.localNodeId().equals(f.coordinatorNodeId())) { + ctx.basicInfo(f.coordinatorNodeId(), "GridDhtTxFinishFuture " + + "waiting for mvcc coordinator reply [mvccCrdNode=" + f.coordinatorNodeId() + + ", loc=" + f.coordinatorNodeId().equals(cctx.localNodeId()) + ']'); - return; + return; + } } } } @@ -592,9 +644,20 @@ private boolean finish(boolean commit, Collection futs = F.viewReadOnly(futures(), new C1, String>() { @SuppressWarnings("unchecked") @Override public String apply(IgniteInternalFuture f) { - return "[node=" + ((MiniFuture)f).node().id() + - ", loc=" + ((MiniFuture)f).node().isLocal() + - ", done=" + f.isDone() + "]"; + if (f.getClass() == MiniFuture.class) { + return "[node=" + ((MiniFuture)f).node().id() + + ", loc=" + ((MiniFuture)f).node().isLocal() + + ", done=" + f.isDone() + "]"; + } + else if (f instanceof MvccFuture) { + MvccFuture crdFut = (MvccFuture)f; + + return "[mvccCrdNode=" + crdFut.coordinatorNodeId() + + ", loc=" + crdFut.coordinatorNodeId().equals(cctx.localNodeId()) + + ", done=" + f.isDone() + "]"; + } + else + return f.toString(); } }); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java index 823b5fee5ed9c..2696c0e0621d4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java @@ -20,11 +20,14 @@ import java.io.Externalizable; import java.nio.ByteBuffer; import java.util.Collection; +import java.util.Map; import java.util.UUID; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.internal.GridDirectCollection; +import org.apache.ignite.internal.GridDirectMap; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishRequest; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -67,6 +70,12 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest { /** One phase commit write version. */ private GridCacheVersion writeVer; + /** */ + private MvccSnapshot mvccSnapshot; + + /** Map of update counters made by this tx. Mapping: cacheId -> partId -> updCntr. */ + @GridDirectMap(keyType = Integer.class, valueType = GridDhtPartitionsUpdateCountersMap.class) + private Map updCntrs; /** * Empty constructor required for {@link Externalizable}. */ @@ -122,7 +131,8 @@ public GridDhtTxFinishRequest( int taskNameHash, boolean addDepInfo, boolean retVal, - boolean waitRemoteTxs + boolean waitRemoteTxs, + MvccSnapshot mvccSnapshot ) { super( xidVer, @@ -151,6 +161,7 @@ public GridDhtTxFinishRequest( this.nearNodeId = nearNodeId; this.isolation = isolation; this.miniId = miniId; + this.mvccSnapshot = mvccSnapshot; needReturnValue(retVal); waitRemoteTransactions(waitRemoteTxs); @@ -207,7 +218,9 @@ public GridDhtTxFinishRequest( boolean addDepInfo, Collection updateIdxs, boolean retVal, - boolean waitRemoteTxs + boolean waitRemoteTxs, + MvccSnapshot mvccSnapshot, + Map updCntrs ) { this(nearNodeId, futId, @@ -232,7 +245,8 @@ public GridDhtTxFinishRequest( taskNameHash, addDepInfo, retVal, - waitRemoteTxs); + waitRemoteTxs, + mvccSnapshot); if (updateIdxs != null && !updateIdxs.isEmpty()) { partUpdateCnt = new GridLongList(updateIdxs.size()); @@ -240,6 +254,15 @@ public GridDhtTxFinishRequest( for (Long idx : updateIdxs) partUpdateCnt.add(idx); } + + this.updCntrs = updCntrs; + } + + /** + * @return Counter. + */ + public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; } /** @@ -339,6 +362,12 @@ public boolean needReturnValue() { public void needReturnValue(boolean retVal) { setFlag(retVal, NEED_RETURN_VALUE_FLAG_MASK); } + /** + * @return Partition update counters map. + */ + public Map updateCountersMap() { + return updCntrs; + } /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { @@ -368,24 +397,36 @@ public void needReturnValue(boolean retVal) { writer.incrementState(); case 23: - if (!writer.writeUuid("nearNodeId", nearNodeId)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 24: - if (!writer.writeMessage("partUpdateCnt", partUpdateCnt)) + if (!writer.writeUuid("nearNodeId", nearNodeId)) return false; writer.incrementState(); case 25: - if (!writer.writeCollection("pendingVers", pendingVers, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("partUpdateCnt", partUpdateCnt)) return false; writer.incrementState(); case 26: + if (!writer.writeCollection("pendingVers", pendingVers, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + case 27: + if (!writer.writeMap("updCntrs", updCntrs, MessageCollectionItemType.INT, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + case 28: if (!writer.writeMessage("writeVer", writeVer)) return false; @@ -428,7 +469,7 @@ public void needReturnValue(boolean retVal) { reader.incrementState(); case 23: - nearNodeId = reader.readUuid("nearNodeId"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -436,7 +477,7 @@ public void needReturnValue(boolean retVal) { reader.incrementState(); case 24: - partUpdateCnt = reader.readMessage("partUpdateCnt"); + nearNodeId = reader.readUuid("nearNodeId"); if (!reader.isLastRead()) return false; @@ -444,7 +485,7 @@ public void needReturnValue(boolean retVal) { reader.incrementState(); case 25: - pendingVers = reader.readCollection("pendingVers", MessageCollectionItemType.MSG); + partUpdateCnt = reader.readMessage("partUpdateCnt"); if (!reader.isLastRead()) return false; @@ -452,6 +493,22 @@ public void needReturnValue(boolean retVal) { reader.incrementState(); case 26: + pendingVers = reader.readCollection("pendingVers", MessageCollectionItemType.MSG); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 27: + updCntrs = reader.readMap("updCntrs", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 28: writeVer = reader.readMessage("writeVer"); if (!reader.isLastRead()) @@ -471,7 +528,7 @@ public void needReturnValue(boolean retVal) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 27; + return 29; } /** {@inheritDoc} */ 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 2e19df2911509..a091d44ac423f 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 @@ -160,7 +160,6 @@ public GridDhtTxLocal( assert nearNodeId != null; assert nearFutId != null; - assert nearMiniId != 0; assert nearXidVer != null; this.nearNodeId = nearNodeId; @@ -424,10 +423,10 @@ private void finishTx(boolean commit, @Nullable IgniteInternalFuture prepFut, Gr final IgniteInternalFuture lockFut = tryRollbackAsync(); if (lockFut != null) { - if (lockFut instanceof GridDhtLockFuture) - ((GridDhtLockFuture)lockFut).onError(rollbackException()); - else { - /** + if (lockFut instanceof DhtLockFuture) + ((DhtLockFuture)lockFut).onError(rollbackException()); + else if (!lockFut.isDone()) { + /* * Prevents race with {@link GridDhtTransactionalCacheAdapter#lockAllAsync * (GridCacheContext, ClusterNode, GridNearLockRequest, CacheEntryPredicate[])} */ @@ -546,6 +545,8 @@ public void rollbackDhtLocal() throws IgniteCheckedException { public IgniteInternalFuture rollbackDhtLocalAsync() { final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, false); + rollbackFuture(fut); + cctx.mvcc().addFuture(fut, fut.futureId()); GridDhtTxPrepareFuture prepFut = this.prepFut; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index 604fe0655801e..11b46a0f2fab8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; @@ -59,7 +60,6 @@ import org.apache.ignite.transactions.TransactionIsolation; import org.apache.ignite.transactions.TransactionState; import org.jetbrains.annotations.Nullable; -import java.util.concurrent.ConcurrentHashMap; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.NOOP; import static org.apache.ignite.transactions.TransactionState.COMMITTED; @@ -78,7 +78,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter { private static final long serialVersionUID = 0L; /** Asynchronous rollback marker for lock futures. */ - protected static final IgniteInternalFuture ROLLBACK_FUT = new GridFutureAdapter<>(); + public static final IgniteInternalFuture ROLLBACK_FUT = new GridFutureAdapter<>(); /** Lock future updater. */ private static final AtomicReferenceFieldUpdater LOCK_FUT_UPD = @@ -108,7 +108,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter { /** Enlist or lock future what is currently in progress. */ @SuppressWarnings("UnusedDeclaration") @GridToStringExclude - protected volatile IgniteInternalFuture lockFut; + protected volatile IgniteInternalFuture lockFut; /** * Empty constructor required for {@link Externalizable}. @@ -852,12 +852,18 @@ public final boolean commitOnPrepare() { return onePhaseCommit() && !near() && !nearOnOriginatingNode; } + /** + * @return Lock future. + */ + public IgniteInternalFuture lockFuture() { + return lockFut; + } + /** * Atomically updates lock future. * * @param oldFut Old future. * @param newFut New future. - * * @return {@code true} If future was changed. */ public boolean updateLockFuture(IgniteInternalFuture oldFut, IgniteInternalFuture newFut) { @@ -870,20 +876,21 @@ public boolean updateLockFuture(IgniteInternalFuture oldFut, IgniteInternalFu * @param cond Clear lock condition. */ public void clearLockFuture(@Nullable IgniteInternalFuture cond) { - IgniteInternalFuture f = lockFut; + while (true) { + IgniteInternalFuture f = lockFut; - if (cond != null && f != cond) - return; - - lockFut = null; + if (f == null + || f == ROLLBACK_FUT + || (cond != null && f != cond) + || updateLockFuture(f, null)) + return; + } } /** - * * @param f Future to finish. * @param err Error. * @param clearLockFut {@code True} if need to clear lock future. - * * @return Finished future. */ public GridFutureAdapter finishFuture(GridFutureAdapter f, Throwable err, boolean clearLockFut) { @@ -901,16 +908,13 @@ public GridFutureAdapter finishFuture(GridFutureAdapter f, Throwable e * @return Current lock future or null if it's safe to roll back. */ public @Nullable IgniteInternalFuture tryRollbackAsync() { - IgniteInternalFuture fut; - - while(true) { - fut = lockFut; - - if (fut != null) - return fut == ROLLBACK_FUT ? null : fut; + while (true) { + final IgniteInternalFuture fut = lockFut; - if (updateLockFuture(null, ROLLBACK_FUT)) + if (fut == ROLLBACK_FUT) return null; + else if (updateLockFuture(fut, ROLLBACK_FUT)) + return fut; } } @@ -924,8 +928,7 @@ protected final IgniteInternalFuture chainOnePhasePre if (commitOnPrepare()) { return finishFuture().chain(new CX1, GridNearTxPrepareResponse>() { @Override public GridNearTxPrepareResponse applyx(IgniteInternalFuture finishFut) - throws IgniteCheckedException - { + throws IgniteCheckedException { return prepFut.get(); } }); 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 d8e204a159487..8ac5419b9f4e2 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 @@ -51,16 +51,21 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; -import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -85,6 +90,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFutureCancelledException; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteReducer; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.thread.IgniteThread; @@ -255,6 +261,11 @@ public GridDhtTxPrepareFuture( timeoutObj = timeout > 0 ? new PrepareTimeoutObject(timeout) : null; } + /** {@inheritDoc} */ + @Nullable @Override public IgniteLogger logger() { + return log; + } + /** {@inheritDoc} */ @Override public IgniteUuid futureId() { return futId; @@ -394,7 +405,8 @@ private void onEntriesLocked() { entryProc, tx.resolveTaskName(), null, - keepBinary); + keepBinary, + null); // TODO IGNITE-7371 if (retVal || txEntry.op() == TRANSFORM) { if (!F.isEmpty(txEntry.entryProcessors())) { @@ -507,7 +519,8 @@ else if (retVal) /*transformClo*/null, /*taskName*/null, /*expiryPlc*/null, - /*keepBinary*/true); + /*keepBinary*/true, + null); // TODO IGNITE-7371 } if (oldVal != null) @@ -886,6 +899,8 @@ private GridNearTxPrepareResponse createPrepareResponse(@Nullable Throwable prep tx.onePhaseCommit(), tx.activeCachesDeploymentEnabled()); + res.mvccSnapshot(tx.mvccSnapshot()); + if (prepErr == null) { if (tx.needReturnValue() || tx.nearOnOriginatingNode() || tx.hasInterceptor()) addDhtValues(res); @@ -1025,7 +1040,7 @@ public void complete() { public void prepare(GridNearTxPrepareRequest req) { assert req != null; - if (tx.empty()) { + if (tx.empty() && !req.queryUpdate()) { tx.setRollbackOnly(); onDone((GridNearTxPrepareResponse)null); @@ -1205,6 +1220,8 @@ private IgniteTxOptimisticCheckedException versionCheckError(IgniteTxEntry entry * */ private void prepare0() { + boolean skipInit = false; + try { if (tx.serializable() && tx.optimistic()) { IgniteCheckedException err0; @@ -1239,6 +1256,29 @@ private void prepare0() { } } + IgniteInternalFuture waitCrdCntrFut = null; + + if (req.requestMvccCounter()) { + assert last; + + assert tx.txState().mvccEnabled(cctx); + + try { + // Request snapshot locally only because + // Mvcc Coordinator is expected to be local. + MvccSnapshot snapshot = cctx.coordinators().tryRequestSnapshotLocal(tx); + + assert snapshot != null : tx.topologyVersion(); + + tx.mvccSnapshot(snapshot); + } + catch (ClusterTopologyCheckedException e) { + onDone(e); + + return; + } + } + onEntriesLocked(); // We are holding transaction-level locks for entries here, so we can get next write version. @@ -1258,11 +1298,37 @@ private void prepare0() { if (isDone()) return; - if (last) - sendPrepareRequests(); + if (last) { + if (waitCrdCntrFut != null) { + skipInit = true; + + waitCrdCntrFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + fut.get(); + + sendPrepareRequests(); + + markInitialized(); + } + catch (Throwable e) { + U.error(log, "Failed to get mvcc version for tx [txId=" + tx.nearXidVersion() + + ", err=" + e + ']', e); + + GridNearTxPrepareResponse res = createPrepareResponse(e); + + onDone(res, res.error()); + } + } + }); + } + else + sendPrepareRequests(); + } } finally { - markInitialized(); + if (!skipInit) + markInitialized(); } } @@ -1280,15 +1346,23 @@ private void sendPrepareRequests() { } } + assert !tx.txState().mvccEnabled(cctx) || !tx.onePhaseCommit() || tx.mvccSnapshot() != null; + int miniId = 0; assert tx.transactionNodes() != null; final long timeout = timeoutObj != null ? timeoutObj.timeout : 0; + // Do not need process active transactions on backups. + MvccSnapshot mvccSnapshot = tx.mvccSnapshot(); + + if (mvccSnapshot != null) + mvccSnapshot = mvccSnapshot.withoutActiveTransactions(); + // Create mini futures. for (GridDistributedTxMapping dhtMapping : tx.dhtMap().values()) { - assert !dhtMapping.empty(); + assert !dhtMapping.empty() || dhtMapping.queryUpdate(); ClusterNode n = dhtMapping.primary(); @@ -1300,7 +1374,7 @@ private void sendPrepareRequests() { Collection dhtWrites = dhtMapping.writes(); - if (F.isEmpty(dhtWrites) && F.isEmpty(nearWrites)) + if (!dhtMapping.queryUpdate() && F.isEmpty(dhtWrites) && F.isEmpty(nearWrites)) continue; MiniFuture fut = new MiniFuture(n.id(), ++miniId, dhtMapping, nearMapping); @@ -1325,7 +1399,10 @@ private void sendPrepareRequests() { tx.taskNameHash(), tx.activeCachesDeploymentEnabled(), tx.storeWriteThrough(), - retVal); + retVal, + mvccSnapshot); + + req.queryUpdate(dhtMapping.queryUpdate()); int idx = 0; @@ -1428,7 +1505,8 @@ private void sendPrepareRequests() { tx.taskNameHash(), tx.activeCachesDeploymentEnabled(), tx.storeWriteThrough(), - retVal); + retVal, + mvccSnapshot); for (IgniteTxEntry entry : nearMapping.entries()) { if (CU.writes().apply(entry)) { @@ -1799,7 +1877,7 @@ void onResult(GridDhtTxPrepareResponse res) { } } - if (dhtMapping.empty()) { + if (!dhtMapping.queryUpdate() && dhtMapping.empty()) { dhtMap.remove(nodeId); if (log.isDebugEnabled()) @@ -1830,6 +1908,10 @@ void onResult(GridDhtTxPrepareResponse res) { try { if (entry.initialValue(info.value(), info.version(), + cacheCtx.mvccEnabled() ? ((MvccVersionAware)info).mvccVersion() : null, + cacheCtx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccVersion() : null, + cacheCtx.mvccEnabled() ? ((MvccVersionAware)info).mvccTxState() : TxState.NA, + cacheCtx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccTxState() : TxState.NA, info.ttl(), info.expireTime(), true, 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 88da7b011ea2c..a71fb6c96e653 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 @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -103,6 +104,9 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest { @GridDirectTransient private List nearWritesCacheMissed; + /** */ + private MvccSnapshot mvccSnapshot; + /** {@code True} if remote tx should skip adding itself to completed versions map on finish. */ private boolean skipCompletedVers; @@ -144,7 +148,8 @@ public GridDhtTxPrepareRequest( int taskNameHash, boolean addDepInfo, boolean storeWriteThrough, - boolean retVal) { + boolean retVal, + MvccSnapshot mvccInfo) { super(tx, timeout, null, @@ -165,6 +170,7 @@ public GridDhtTxPrepareRequest( this.nearXidVer = nearXidVer; this.subjId = subjId; this.taskNameHash = taskNameHash; + this.mvccSnapshot = mvccInfo; storeWriteThrough(storeWriteThrough); needReturnValue(retVal); @@ -176,6 +182,13 @@ public GridDhtTxPrepareRequest( skipCompletedVers = tx.xidVersion() == tx.nearXidVersion(); } + /** + * @return Mvcc info. + */ + public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + /** * @return Near cache writes for which cache was not found (possible if client near cache was closed). */ @@ -414,60 +427,66 @@ public boolean skipCompletedVersion() { writer.incrementState(); case 23: - if (!writer.writeUuid("nearNodeId", nearNodeId)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 24: - if (!writer.writeCollection("nearWrites", nearWrites, MessageCollectionItemType.MSG)) + if (!writer.writeUuid("nearNodeId", nearNodeId)) return false; writer.incrementState(); case 25: - if (!writer.writeMessage("nearXidVer", nearXidVer)) + if (!writer.writeCollection("nearWrites", nearWrites, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 26: - if (!writer.writeCollection("ownedKeys", ownedKeys, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("nearXidVer", nearXidVer)) return false; writer.incrementState(); case 27: - if (!writer.writeCollection("ownedVals", ownedVals, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("ownedKeys", ownedKeys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 28: - if (!writer.writeBitSet("preloadKeys", preloadKeys)) + if (!writer.writeCollection("ownedVals", ownedVals, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 29: - if (!writer.writeBoolean("skipCompletedVers", skipCompletedVers)) + if (!writer.writeBitSet("preloadKeys", preloadKeys)) return false; writer.incrementState(); case 30: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeBoolean("skipCompletedVers", skipCompletedVers)) return false; writer.incrementState(); case 31: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 32: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 33: if (!writer.writeMessage("topVer", topVer)) return false; @@ -514,7 +533,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 23: - nearNodeId = reader.readUuid("nearNodeId"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -522,7 +541,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 24: - nearWrites = reader.readCollection("nearWrites", MessageCollectionItemType.MSG); + nearNodeId = reader.readUuid("nearNodeId"); if (!reader.isLastRead()) return false; @@ -530,7 +549,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 25: - nearXidVer = reader.readMessage("nearXidVer"); + nearWrites = reader.readCollection("nearWrites", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -538,7 +557,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 26: - ownedKeys = reader.readCollection("ownedKeys", MessageCollectionItemType.MSG); + nearXidVer = reader.readMessage("nearXidVer"); if (!reader.isLastRead()) return false; @@ -546,7 +565,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 27: - ownedVals = reader.readCollection("ownedVals", MessageCollectionItemType.MSG); + ownedKeys = reader.readCollection("ownedKeys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -554,7 +573,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 28: - preloadKeys = reader.readBitSet("preloadKeys"); + ownedVals = reader.readCollection("ownedVals", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -562,7 +581,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 29: - skipCompletedVers = reader.readBoolean("skipCompletedVers"); + preloadKeys = reader.readBitSet("preloadKeys"); if (!reader.isLastRead()) return false; @@ -570,7 +589,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 30: - subjId = reader.readUuid("subjId"); + skipCompletedVers = reader.readBoolean("skipCompletedVers"); if (!reader.isLastRead()) return false; @@ -578,7 +597,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 31: - taskNameHash = reader.readInt("taskNameHash"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -586,6 +605,14 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 32: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 33: topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) @@ -605,7 +632,7 @@ public boolean skipCompletedVersion() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 33; + return 34; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java new file mode 100644 index 0000000000000..dd3085500e1e6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java @@ -0,0 +1,140 @@ +/* + * 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.Objects; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.GridQueryCancel; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; + +/** + * Cache lock future. + */ +public final class GridDhtTxQueryEnlistFuture extends GridDhtTxAbstractEnlistFuture { + /** Involved cache ids. */ + private final int[] cacheIds; + + /** Schema name. */ + private final String schema; + + /** Query string. */ + private final String qry; + + /** Query parameters. */ + private final Object[] params; + + /** Flags. */ + private final int flags; + + /** Fetch page size. */ + private final int pageSize; + + /** + * @param nearNodeId Near node ID. + * @param nearLockVer Near lock version. + * @param mvccSnapshot Mvcc snapshot. + * @param threadId Thread ID. + * @param nearFutId Near future id. + * @param nearMiniId Near mini future id. + * @param tx Transaction. + * @param cacheIds Involved cache ids. + * @param parts Partitions. + * @param schema Schema name. + * @param qry Query string. + * @param params Query parameters. + * @param flags Flags. + * @param pageSize Fetch page size. + * @param timeout Lock acquisition timeout. + * @param cctx Cache context. + */ + public GridDhtTxQueryEnlistFuture( + UUID nearNodeId, + GridCacheVersion nearLockVer, + MvccSnapshot mvccSnapshot, + long threadId, + IgniteUuid nearFutId, + int nearMiniId, + GridDhtTxLocalAdapter tx, + int[] cacheIds, + int[] parts, + String schema, + String qry, + Object[] params, + int flags, + int pageSize, + long timeout, + GridCacheContext cctx) { + super(nearNodeId, + nearLockVer, + mvccSnapshot, + threadId, + nearFutId, + nearMiniId, + parts, + tx, + timeout, + cctx); + + assert timeout >= 0; + assert nearNodeId != null; + assert nearLockVer != null; + assert threadId == tx.threadId(); + + this.cacheIds = cacheIds; + this.schema = schema; + this.qry = qry; + this.params = params; + this.flags = flags; + this.pageSize = pageSize; + } + + /** {@inheritDoc} */ + @Override protected UpdateSourceIterator createIterator() throws IgniteCheckedException { + return cctx.kernalContext().query().prepareDistributedUpdate(cctx, cacheIds, parts, schema, qry, + params, flags, pageSize, 0, tx.topologyVersionSnapshot(), mvccSnapshot, new GridQueryCancel()); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + GridDhtTxQueryEnlistFuture future = (GridDhtTxQueryEnlistFuture)o; + + return Objects.equals(futId, future.futId); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return futId.hashCode(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtTxQueryEnlistFuture.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java new file mode 100644 index 0000000000000..650e1dc73240e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistRequest.java @@ -0,0 +1,404 @@ +/* + * 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.nio.ByteBuffer; +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectCollection; +import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * + */ +public class GridDhtTxQueryEnlistRequest extends GridCacheIdMessage { + /** */ + private static final long serialVersionUID = 5103887309729425173L; + + /** */ + private IgniteUuid dhtFutId; + + /** */ + private int batchId; + + /** DHT tx version. */ + private GridCacheVersion lockVer; + + /** */ + private EnlistOperation op; + + /** */ + private int mvccOpCnt; + + /** */ + @GridDirectCollection(KeyCacheObject.class) + private List keys; + + /** */ + @GridDirectCollection(Message.class) + private List vals; + + /** */ + private GridLongList updCntrs; + + /** + * + */ + public GridDhtTxQueryEnlistRequest() { + } + + /** + * @param cacheId Cache id. + * @param dhtFutId DHT future id. + * @param lockVer Lock version. + * @param op Operation. + * @param batchId Batch id. + * @param mvccOpCnt Mvcc operation counter. + * @param keys Keys. + * @param vals Values. + * @param updCntrs Update counters. + */ + GridDhtTxQueryEnlistRequest(int cacheId, + IgniteUuid dhtFutId, + GridCacheVersion lockVer, + EnlistOperation op, + int batchId, + int mvccOpCnt, + List keys, + List vals, + GridLongList updCntrs) { + this.cacheId = cacheId; + this.dhtFutId = dhtFutId; + this.lockVer = lockVer; + this.op = op; + this.batchId = batchId; + this.mvccOpCnt = mvccOpCnt; + this.keys = keys; + this.vals = vals; + this.updCntrs = updCntrs; + } + + /** + * Returns request rows number. + * + * @return Request rows number. + */ + public int batchSize() { + return keys == null ? 0 : keys.size(); + } + + /** + * @return Dht future id. + */ + public IgniteUuid dhtFutureId() { + return dhtFutId; + } + + /** + * @return Lock version. + */ + public GridCacheVersion version() { + return lockVer; + } + + /** + * @return Mvcc operation counter. + */ + public int operationCounter() { + return mvccOpCnt; + } + + /** + * @return Operation. + */ + public EnlistOperation op() { + return op; + } + + /** + * @return Keys. + */ + public List keys() { + return keys; + } + + /** + * @return Values. + */ + public List values() { + return vals; + } + + /** + * @return Update counters. + */ + public GridLongList updateCounters() { + return updCntrs; + } + + /** + * @return Batch id. + */ + public int batchId() { + return batchId; + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return false; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 155; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + CacheObjectContext objCtx = ctx.cacheContext(cacheId).cacheObjectContext(); + + if (keys != null) { + for (int i = 0; i < keys.size(); i++) { + + keys.get(i).prepareMarshal(objCtx); + + if (vals != null) { + Message val = vals.get(i); + + if (val instanceof CacheObject) + ((CacheObject)val).prepareMarshal(objCtx); + else if (val instanceof CacheEntryInfoCollection) { + for (GridCacheEntryInfo entry : ((CacheEntryInfoCollection)val).infos()) { + CacheObject entryVal = entry.value(); + + if (entryVal != null) + entryVal.prepareMarshal(objCtx); + } + } + } + } + } + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + CacheObjectContext objCtx = ctx.cacheContext(cacheId).cacheObjectContext(); + + if (keys != null) { + for (int i = 0; i < keys.size(); i++) { + keys.get(i).finishUnmarshal(objCtx, ldr); + + if (vals != null) { + Message val = vals.get(i); + + if (val instanceof CacheObject) + ((CacheObject)val).finishUnmarshal(objCtx, ldr); + else if (val instanceof CacheEntryInfoCollection) { + for (GridCacheEntryInfo entry : ((CacheEntryInfoCollection)val).infos()) { + CacheObject entryVal = entry.value(); + + if (entryVal != null) + entryVal.finishUnmarshal(objCtx, ldr); + } + } + } + } + } + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeInt("batchId", batchId)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeIgniteUuid("dhtFutId", dhtFutId)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeMessage("lockVer", lockVer)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeInt("mvccOpCnt", mvccOpCnt)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1)) + return false; + + writer.incrementState(); + + case 9: + if (!writer.writeMessage("updCntrs", updCntrs)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + batchId = reader.readInt("batchId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + dhtFutId = reader.readIgniteUuid("dhtFutId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + keys = reader.readCollection("keys", MessageCollectionItemType.MSG); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + lockVer = reader.readMessage("lockVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + mvccOpCnt = reader.readInt("mvccOpCnt"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: + byte opOrd; + + opOrd = reader.readByte("op"); + + if (!reader.isLastRead()) + return false; + + op = EnlistOperation.fromOrdinal(opOrd); + + reader.incrementState(); + + case 9: + updCntrs = reader.readMessage("updCntrs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: + vals = reader.readCollection("vals", MessageCollectionItemType.MSG); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridDhtTxQueryEnlistRequest.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 11; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtTxQueryEnlistRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java new file mode 100644 index 0000000000000..411848958d4f4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistResponse.java @@ -0,0 +1,205 @@ +/* + * 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.nio.ByteBuffer; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +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.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * + */ +public class GridDhtTxQueryEnlistResponse extends GridCacheIdMessage { + /** */ + private static final long serialVersionUID = -1510546400896574705L; + + /** Future ID. */ + private IgniteUuid futId; + + /** */ + private int batchId; + + /** Error. */ + @GridDirectTransient + private Throwable err; + + /** Serialized error. */ + private byte[] errBytes; + + /** + * + */ + public GridDhtTxQueryEnlistResponse() { + } + + /** + * @param cacheId Cache id. + * @param futId Future id. + * @param batchId Batch id. + * @param err Error. + */ + GridDhtTxQueryEnlistResponse(int cacheId, IgniteUuid futId, int batchId, + Throwable err) { + this.cacheId = cacheId; + this.futId = futId; + this.batchId = batchId; + this.err = err; + } + + /** + * @return Future id. + */ + public IgniteUuid futureId() { + return futId; + } + + /** + * @return Batch id. + */ + public int batchId() { + return batchId; + } + + /** + * @return Error. + */ + public Throwable error() { + return err; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + if (err != null && errBytes == null) + errBytes = U.marshal(ctx.marshaller(), err); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + if (errBytes != null) + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return false; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 144; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 6; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeInt("batchId", batchId)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeByteArray("errBytes", errBytes)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeIgniteUuid("futId", futId)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + batchId = reader.readInt("batchId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + errBytes = reader.readByteArray("errBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + futId = reader.readIgniteUuid("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridDhtTxQueryEnlistResponse.class); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtTxQueryEnlistResponse.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java new file mode 100644 index 0000000000000..5df1d5843662a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryFirstEnlistRequest.java @@ -0,0 +1,370 @@ +/* + * 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.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotWithoutTxs; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * First enlist request. + */ +public class GridDhtTxQueryFirstEnlistRequest extends GridDhtTxQueryEnlistRequest { + /** */ + private static final long serialVersionUID = -7494735627739420176L; + + /** Tx initiator. Primary node in case of remote DHT tx. */ + private UUID subjId; + + /** */ + private AffinityTopologyVersion topVer; + + /** */ + private long crdVer; + + /** */ + private long cntr; + + /** */ + private long cleanupVer; + + /** */ + private long timeout; + + /** */ + private int taskNameHash; + + /** */ + private UUID nearNodeId; + + /** Near tx version. */ + private GridCacheVersion nearXidVer; + + /** + * + */ + public GridDhtTxQueryFirstEnlistRequest() { + } + + /** + * @param cacheId Cache id. + * @param dhtFutId DHT future id. + * @param subjId Subject id. + * @param topVer Topology version. + * @param lockVer Lock version. + * @param snapshot Mvcc snapshot. + * @param timeout Timeout. + * @param taskNameHash Task name hash. + * @param nearNodeId Near node id. + * @param nearXidVer Near xid version. + * @param op Operation. + * @param batchId Batch id. + * @param keys Keys. + * @param vals Values. + * @param updCntrs Update counters. + */ + GridDhtTxQueryFirstEnlistRequest(int cacheId, + IgniteUuid dhtFutId, + UUID subjId, + AffinityTopologyVersion topVer, + GridCacheVersion lockVer, + MvccSnapshot snapshot, + long timeout, + int taskNameHash, + UUID nearNodeId, + GridCacheVersion nearXidVer, + EnlistOperation op, + int batchId, + List keys, + List vals, + GridLongList updCntrs) { + super(cacheId, dhtFutId, lockVer, op, batchId, snapshot.operationCounter(), keys, vals, updCntrs); + this.cacheId = cacheId; + this.subjId = subjId; + this.topVer = topVer; + this.crdVer = snapshot.coordinatorVersion(); + this.cntr = snapshot.counter(); + this.cleanupVer = snapshot.cleanupVersion(); + this.timeout = timeout; + this.taskNameHash = taskNameHash; + this.nearNodeId = nearNodeId; + this.nearXidVer = nearXidVer; + } + + /** {@inheritDoc} */ + @Override public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** + * @return Near node id. + */ + public UUID nearNodeId() { + return nearNodeId; + } + + /** + * @return Near transaction ID. + */ + public GridCacheVersion nearXidVersion() { + return nearXidVer; + } + + /** + * @return Max lock wait time. + */ + public long timeout() { + return timeout; + } + + /** + * @return Subject id. + */ + public UUID subjectId() { + return subjId; + } + + /** + * @return Task name hash. + */ + public int taskNameHash() { + return taskNameHash; + } + + /** + * @return MVCC snapshot. + */ + public MvccSnapshot mvccSnapshot() { + return new MvccSnapshotWithoutTxs(crdVer, cntr, operationCounter(), cleanupVer); + } + + /** + * @return Coordinator version. + */ + public long coordinatorVersion() { + return crdVer; + } + + /** + * @return Counter. + */ + public long counter() { + return cntr; + } + + /** + * @return Cleanup version. + */ + public long cleanupVersion() { + return cleanupVer; + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return false; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 156; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 11: + if (!writer.writeLong("cleanupVer", cleanupVer)) + return false; + + writer.incrementState(); + + case 12: + if (!writer.writeLong("cntr", cntr)) + return false; + + writer.incrementState(); + + case 13: + if (!writer.writeLong("crdVer", crdVer)) + return false; + + writer.incrementState(); + + case 14: + if (!writer.writeUuid("nearNodeId", nearNodeId)) + return false; + + writer.incrementState(); + + case 15: + if (!writer.writeMessage("nearXidVer", nearXidVer)) + return false; + + writer.incrementState(); + + case 16: + if (!writer.writeUuid("subjId", subjId)) + return false; + + writer.incrementState(); + + case 17: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 18: + if (!writer.writeLong("timeout", timeout)) + return false; + + writer.incrementState(); + + case 19: + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 11: + cleanupVer = reader.readLong("cleanupVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 12: + cntr = reader.readLong("cntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 13: + crdVer = reader.readLong("crdVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 14: + nearNodeId = reader.readUuid("nearNodeId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 15: + nearXidVer = reader.readMessage("nearXidVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 16: + subjId = reader.readUuid("subjId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 17: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 18: + timeout = reader.readLong("timeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 19: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridDhtTxQueryFirstEnlistRequest.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 20; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtTxQueryFirstEnlistRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryResultsEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryResultsEnlistFuture.java new file mode 100644 index 0000000000000..6169e1fe7fc37 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryResultsEnlistFuture.java @@ -0,0 +1,130 @@ +/* + * 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.Collection; +import java.util.Iterator; +import java.util.Objects; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; + +/** + * Future processing transaction enlisting and locking of entries + * produces by complex DML queries with reduce step. + */ +public final class GridDhtTxQueryResultsEnlistFuture extends GridDhtTxAbstractEnlistFuture implements UpdateSourceIterator { + /** */ + private static final long serialVersionUID = -4933550335145438798L; + /** */ + private EnlistOperation op; + + /** */ + private Iterator it; + + /** + * @param nearNodeId Near node ID. + * @param nearLockVer Near lock version. + * @param mvccSnapshot Mvcc snapshot. + * @param threadId Thread ID. + * @param nearFutId Near future id. + * @param nearMiniId Near mini future id. + * @param tx Transaction. + * @param timeout Lock acquisition timeout. + * @param cctx Cache context. + * @param rows Collection of rows. + * @param op Operation. + */ + public GridDhtTxQueryResultsEnlistFuture(UUID nearNodeId, + GridCacheVersion nearLockVer, + MvccSnapshot mvccSnapshot, + long threadId, + IgniteUuid nearFutId, + int nearMiniId, + GridDhtTxLocalAdapter tx, + long timeout, + GridCacheContext cctx, + Collection rows, + EnlistOperation op) { + super(nearNodeId, + nearLockVer, + mvccSnapshot, + threadId, + nearFutId, + nearMiniId, + null, + tx, + timeout, + cctx); + + this.op = op; + + it = rows.iterator(); + + skipNearNodeUpdates = true; + } + + /** {@inheritDoc} */ + @Override protected UpdateSourceIterator createIterator() throws IgniteCheckedException { + return this; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + GridDhtTxQueryResultsEnlistFuture future = (GridDhtTxQueryResultsEnlistFuture)o; + + return Objects.equals(futId, future.futId); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return futId.hashCode(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtTxQueryResultsEnlistFuture.class, this); + } + + /** {@inheritDoc} */ + @Override public EnlistOperation operation() { + return op; + } + + /** {@inheritDoc} */ + public boolean hasNextX() { + return it.hasNext(); + } + + /** {@inheritDoc} */ + public Object nextX() { + return it.next(); + } +} 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 746eb387a9beb..08ecf28e3a3b0 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 @@ -21,26 +21,38 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.UUID; import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterTopologyException; +import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteSingleStateImpl; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteStateImpl; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.tostring.GridToStringBuilder; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; @@ -67,6 +79,8 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter { /** Store write through flag. */ private boolean storeWriteThrough; + /** Map of update counters made by this tx. Mapping: cacheId -> partId -> updCntr. */ + private Map updCntrs; /** * Empty constructor required for {@link Externalizable}. */ @@ -368,6 +382,136 @@ public void addWrite(GridCacheContext cacheCtx, txState.addWriteEntry(key, txEntry); } + /** + * + * @param ctx Cache context. + * @param op Operation. + * @param keys Keys. + * @param vals Values. + * @param snapshot Mvcc snapshot. + * @param updCntrs Update counters. + * @throws IgniteCheckedException If failed. + */ + public void mvccEnlistBatch(GridCacheContext ctx, EnlistOperation op, List keys, + List vals, MvccSnapshot snapshot, GridLongList updCntrs) throws IgniteCheckedException { + assert keys != null && updCntrs != null && keys.size() == updCntrs.size(); + + WALPointer ptr = null; + + GridDhtCacheAdapter dht = ctx.dht(); + + addActiveCache(ctx, false); + + for (int i = 0; i < keys.size(); i++) { + KeyCacheObject key = keys.get(i); + + assert key != null; + + int part = ctx.affinity().partition(key); + + GridDhtLocalPartition locPart = ctx.topology().localPartition(part, topologyVersion(), false); + + if (locPart == null || !locPart.reserve()) + throw new ClusterTopologyException("Can not reserve partition. Please retry on stable topology."); + + try { + CacheObject val = null; + + Message val0 = vals != null ? vals.get(i) : null; + + CacheEntryInfoCollection entries = + val0 instanceof CacheEntryInfoCollection ? (CacheEntryInfoCollection)val0 : null; + + if (entries == null && !op.isDeleteOrLock()) + val = (val0 instanceof CacheObject) ? (CacheObject)val0 : null; + + GridDhtCacheEntry entry = dht.entryExx(key, topologyVersion()); + + GridCacheUpdateTxResult updRes; + + while (true) { + ctx.shared().database().checkpointReadLock(); + + try { + if (entries == null) { + switch (op) { + case DELETE: + updRes = entry.mvccRemove( + this, + ctx.localNodeId(), + topologyVersion(), + updCntrs.get(i), + snapshot, + false); + + break; + + case INSERT: + case UPSERT: + case UPDATE: + updRes = entry.mvccSet( + this, + ctx.localNodeId(), + val, + 0, + topologyVersion(), + updCntrs.get(i), + snapshot, + op.cacheOperation(), + false, + false); + + break; + + default: + throw new IgniteSQLException("Cannot acquire lock for operation [op= " + + op + "]" + "Operation is unsupported at the moment ", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + } + } + else { + updRes = entry.mvccUpdateRowsWithPreloadInfo(this, + ctx.localNodeId(), + topologyVersion(), + updCntrs.get(i), + entries.infos(), + op.cacheOperation(), + snapshot); + } + + break; + } + catch (GridCacheEntryRemovedException ignore) { + entry = dht.entryExx(key); + } + finally { + ctx.shared().database().checkpointReadUnlock(); + } + } + + assert updRes.updateFuture() == null : "Entry should not be locked on the backup"; + + ptr = updRes.loggedPointer(); + } + finally { + locPart.release(); + } + } + + if (ptr != null && !ctx.tm().logTxRecords()) + ctx.shared().wal().flush(ptr, true); + } + + /** {@inheritDoc} */ + @Override public void updateCountersMap(Map updCntrsMap) { + this.updCntrs = updCntrsMap; + } + + /** {@inheritDoc} */ + @Override public Map updateCountersMap() { + return updCntrs; + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(GridDhtTxRemote.class, this, "super", super.toString()); 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 30d36e862fa86..a76844ae6168b 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 @@ -43,6 +43,10 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTrackerImpl; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotResponseListener; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridLeanMap; @@ -64,7 +68,8 @@ /** * Colocated get future. */ -public class GridPartitionedGetFuture extends CacheDistributedGetFutureAdapter { +public class GridPartitionedGetFuture extends CacheDistributedGetFutureAdapter + implements MvccSnapshotResponseListener { /** */ private static final long serialVersionUID = 0L; @@ -74,6 +79,12 @@ public class GridPartitionedGetFuture extends CacheDistributedGetFutureAda /** Logger. */ private static IgniteLogger log; + /** */ + protected final MvccSnapshot mvccSnapshot; + + /** */ + private MvccQueryTracker mvccTracker; + /** * @param cctx Context. * @param keys Keys. @@ -88,6 +99,7 @@ public class GridPartitionedGetFuture extends CacheDistributedGetFutureAda * @param skipVals Skip values flag. * @param needVer If {@code true} returns values as tuples containing value and version. * @param keepCacheObjects Keep cache objects flag. + * @param mvccSnapshot Mvcc snapshot. */ public GridPartitionedGetFuture( GridCacheContext cctx, @@ -101,7 +113,8 @@ public GridPartitionedGetFuture( @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean skipVals, boolean needVer, - boolean keepCacheObjects + boolean keepCacheObjects, + @Nullable MvccSnapshot mvccSnapshot ) { super(cctx, keys, @@ -115,11 +128,31 @@ public GridPartitionedGetFuture( needVer, keepCacheObjects, recovery); + assert mvccSnapshot == null || cctx.mvccEnabled(); + + this.mvccSnapshot = mvccSnapshot; if (log == null) log = U.logger(cctx.kernalContext(), logRef, GridPartitionedGetFuture.class); } + /** + * @return Mvcc snapshot if mvcc is enabled for cache. + */ + @Nullable private MvccSnapshot mvccSnapshot() { + if (!cctx.mvccEnabled()) + return null; + + if (mvccSnapshot != null) + return mvccSnapshot; + + MvccSnapshot snapshot = mvccTracker.snapshot(); + + assert snapshot != null : "[fut=" + this + ", mvccTracker=" + mvccTracker + "]"; + + return snapshot; + } + /** * Initializes future. * @@ -129,16 +162,45 @@ public void init(AffinityTopologyVersion topVer) { AffinityTopologyVersion lockedTopVer = cctx.shared().lockedTopologyVersion(null); if (lockedTopVer != null) { - canRemap = false; + topVer = lockedTopVer; - map(keys, Collections.>emptyMap(), lockedTopVer); + canRemap = false; } else { topVer = topVer.topologyVersion() > 0 ? topVer : canRemap ? cctx.affinity().affinityTopologyVersion() : cctx.shared().exchange().readyAffinityVersion(); + } - map(keys, Collections.>emptyMap(), topVer); + if (!cctx.mvccEnabled() || mvccSnapshot != null) + initialMap(topVer); + else { + mvccTracker = new MvccQueryTrackerImpl(cctx, canRemap); + + trackable = true; + + cctx.mvcc().addFuture(this, futId); + + mvccTracker.requestSnapshot(topVer, this); } + } + + @Override public void onResponse(MvccSnapshot res) { + AffinityTopologyVersion topVer = mvccTracker.topologyVersion(); + + assert topVer != null; + + initialMap(topVer); + } + + @Override public void onError(IgniteCheckedException e) { + onDone(e); + } + + /** + * @param topVer Topology version. + */ + private void initialMap(AffinityTopologyVersion topVer) { + map(keys, Collections.>emptyMap(), topVer); markInitialized(); } @@ -197,10 +259,12 @@ public void init(AffinityTopologyVersion topVer) { /** {@inheritDoc} */ @Override public boolean onDone(Map res, Throwable err) { if (super.onDone(res, err)) { - // Don't forget to clean up. if (trackable) cctx.mvcc().removeFuture(futId); + if (mvccTracker != null) + mvccTracker.onDone(); + cache().sendTtlUpdateRequest(expiryPlc); return true; @@ -293,7 +357,8 @@ private void map( taskName == null ? 0 : taskName.hashCode(), expiryPlc, skipVals, - recovery); + recovery, + mvccSnapshot()); final Collection invalidParts = fut.invalidPartitions(); @@ -350,7 +415,8 @@ private void map( false, skipVals, cctx.deploymentEnabled(), - recovery); + recovery, + mvccSnapshot()); add(fut); // Append new future. @@ -463,7 +529,9 @@ private boolean localGet(AffinityTopologyVersion topVer, KeyCacheObject key, int GridCacheVersion ver = null; if (readNoEntry) { - CacheDataRow row = cctx.offheap().read(cctx, key); + CacheDataRow row = cctx.mvccEnabled() ? + cctx.offheap().mvccRead(cctx, key, mvccSnapshot()) : + cctx.offheap().read(cctx, key); if (row != null) { long expireTime = row.expireTime(); @@ -506,6 +574,7 @@ private boolean localGet(AffinityTopologyVersion topVer, KeyCacheObject key, int taskName, expiryPlc, !deserializeBinary, + mvccSnapshot(), null); if (getRes != null) { @@ -524,7 +593,8 @@ private boolean localGet(AffinityTopologyVersion topVer, KeyCacheObject key, int null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + mvccSnapshot()); } entry.touch(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 83e46ab817497..fd9bc77ba05ca 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 @@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -124,6 +125,9 @@ public class GridPartitionedSingleGetFuture extends GridCacheFutureAdapter invalidParts = fut.invalidPartitions(); @@ -305,7 +313,8 @@ private void map(final AffinityTopologyVersion topVer) { /*add reader*/false, needVer, cctx.deploymentEnabled(), - recovery); + recovery, + mvccSnapshot); try { cctx.io().send(node, req, cctx.ioPolicy()); @@ -379,7 +388,8 @@ private boolean localGet(AffinityTopologyVersion topVer, int part) { boolean skipEntry = readNoEntry; if (readNoEntry) { - CacheDataRow row = cctx.offheap().read(cctx, key); + CacheDataRow row = mvccSnapshot != null ? cctx.offheap().mvccRead(cctx, key, mvccSnapshot) : + cctx.offheap().read(cctx, key); if (row != null) { long expireTime = row.expireTime(); @@ -422,6 +432,7 @@ private boolean localGet(AffinityTopologyVersion topVer, int part) { taskName, expiryPlc, true, + mvccSnapshot, null); if (res != null) { @@ -440,7 +451,8 @@ private boolean localGet(AffinityTopologyVersion topVer, int part) { null, taskName, expiryPlc, - true); + true, + mvccSnapshot); } entry.touch(topVer); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java new file mode 100644 index 0000000000000..bb863fb85eb66 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java @@ -0,0 +1,169 @@ +/* + * 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.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistResponse; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.lang.GridClosureException; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.NotNull; + +/** + * + */ +public final class NearTxQueryEnlistResultHandler implements CI1> { + /** */ + private static final long serialVersionUID = 5189735824793607906L; + + /** */ + private static final NearTxQueryEnlistResultHandler INSTANCE = new NearTxQueryEnlistResultHandler(); + + /** */ + private NearTxQueryEnlistResultHandler() {} + + /** + * @return Handler instance. + */ + public static NearTxQueryEnlistResultHandler instance() { + return INSTANCE; + } + + /** + * @param future Enlist future. + * @return Enlist response. + */ + @SuppressWarnings("unchecked") + public static T createResponse(IgniteInternalFuture future) { + assert future != null; + + Class clazz = future.getClass(); + + if (clazz == GridDhtTxQueryResultsEnlistFuture.class) + return (T)createResponse((GridDhtTxQueryResultsEnlistFuture)future); + else if (clazz == GridDhtTxQueryEnlistFuture.class) + return (T)createResponse((GridDhtTxQueryEnlistFuture)future); + else + throw new IllegalStateException(); + } + + /** + * @param future Enlist future. + * @return Enlist response. + */ + @NotNull private static GridNearTxQueryEnlistResponse createResponse(GridDhtTxQueryEnlistFuture future) { + try { + future.get(); + + assert future.tx.queryEnlisted() || future.cnt == 0; + + return new GridNearTxQueryEnlistResponse(future.cctx.cacheId(), future.nearFutId, future.nearMiniId, + future.nearLockVer, future.cnt, future.tx.empty() && !future.tx.queryEnlisted()); + } + catch (IgniteCheckedException e) { + return new GridNearTxQueryEnlistResponse(future.cctx.cacheId(), future.nearFutId, future.nearMiniId, future.nearLockVer, e); + } + } + + /** + * @param future Enlist future. + * @return Enlist response. + */ + @NotNull private static GridNearTxQueryResultsEnlistResponse createResponse(GridDhtTxQueryResultsEnlistFuture future) { + try { + future.get(); + + GridCacheVersion ver = null; + IgniteUuid id = null; + GridLongList updCntrs = null; + + if (future.hasNearNodeUpdates) { + ver = future.cctx.tm().mappedVersion(future.nearLockVer); + id = future.futId; + updCntrs = future.nearUpdCntrs; + } + + return new GridNearTxQueryResultsEnlistResponse(future.cctx.cacheId(), future.nearFutId, future.nearMiniId, + future.nearLockVer, future.cnt, ver, id, updCntrs); + } + catch (IgniteCheckedException e) { + return new GridNearTxQueryResultsEnlistResponse(future.cctx.cacheId(), future.nearFutId, future.nearMiniId, + future.nearLockVer, e); + } + } + + /** {@inheritDoc} */ + @Override public void apply(IgniteInternalFuture fut0) { + GridDhtTxAbstractEnlistFuture fut = (GridDhtTxAbstractEnlistFuture)fut0; + + GridCacheContext cctx = fut.cctx; + GridDhtTxLocal tx = (GridDhtTxLocal)fut.tx; + UUID nearNodeId = fut.nearNodeId; + + GridNearTxQueryEnlistResponse res = createResponse(fut); + + if (res.removeMapping()) { + // TODO IGNITE-9133 + tx.rollbackDhtLocalAsync().listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut0) { + try { + cctx.io().send(nearNodeId, res, cctx.ioPolicy()); + } + catch (IgniteCheckedException e) { + U.error(fut.log, "Failed to send near enlist response [" + + "tx=" + CU.txString(tx) + + ", node=" + nearNodeId + + ", res=" + res + ']', e); + + throw new GridClosureException(e); + } + } + }); + + return; + } + + try { + cctx.io().send(nearNodeId, res, cctx.ioPolicy()); + } + catch (IgniteCheckedException e) { + U.error(fut.log, "Failed to send near enlist response (will rollback transaction) [" + + "tx=" + CU.txString(tx) + + ", node=" + nearNodeId + + ", res=" + res + ']', e); + + try { + tx.rollbackDhtLocalAsync(); + } + catch (Throwable e1) { + e.addSuppressed(e1); + } + + throw new GridClosureException(e); + } + } +} 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 112b91320f39e..7bc04dc27c08a 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 @@ -65,7 +65,6 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; -import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture; @@ -83,6 +82,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrExpirationInfo; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; @@ -1422,7 +1422,8 @@ private IgniteInternalFuture getAsync0(KeyCacheObject key, skipVals, needVer, false, - recovery); + recovery, + null); fut.init(); @@ -1531,6 +1532,7 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection> getAllAsync0(@Nullable Collection interceptorRes = ctx.config().getInterceptor() .onBeforeRemove(new CacheLazyEntry(ctx, entry.key(), old, req.keepBinary())); 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 eaa93aea14e53..f6de594a04cf5 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.GridCacheReturn; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException; @@ -241,7 +242,8 @@ public GridDistributedCacheEntry entryExx( skipVals, needVer, /*keepCacheObjects*/false, - opCtx != null && opCtx.recovery()); + opCtx != null && opCtx.recovery(), + null); fut.init(); @@ -319,7 +321,7 @@ public GridDistributedCacheEntry entryExx( * @param needVer Need version. * @return Loaded values. */ - public IgniteInternalFuture> loadAsync( + private IgniteInternalFuture> loadAsync( @Nullable Collection keys, boolean readThrough, boolean forcePrimary, @@ -341,7 +343,8 @@ public IgniteInternalFuture> loadAsync( expiryPlc, skipVals, needVer, - false); + false, + null); } /** @@ -370,7 +373,8 @@ public final IgniteInternalFuture loadAsync( boolean skipVals, boolean needVer, boolean keepCacheObj, - boolean recovery + boolean recovery, + @Nullable MvccSnapshot mvccSnapshot ) { GridPartitionedSingleGetFuture fut = new GridPartitionedSingleGetFuture(ctx, ctx.toCacheKeyObject(key), @@ -384,7 +388,8 @@ public final IgniteInternalFuture loadAsync( skipVals, needVer, keepCacheObj, - recovery); + recovery, + mvccSnapshot); fut.init(); @@ -403,6 +408,7 @@ public final IgniteInternalFuture loadAsync( * @param skipVals Skip values flag. * @param needVer If {@code true} returns values as tuples containing value and version. * @param keepCacheObj Keep cache objects flag. + * @param mvccSnapshot Mvcc snapshot. * @return Load future. */ public final IgniteInternalFuture> loadAsync( @@ -417,8 +423,11 @@ public final IgniteInternalFuture> loadAsync( @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean skipVals, boolean needVer, - boolean keepCacheObj + boolean keepCacheObj, + @Nullable MvccSnapshot mvccSnapshot ) { + assert mvccSnapshot == null || ctx.mvccEnabled(); + if (keys == null || keys.isEmpty()) return new GridFinishedFuture<>(Collections.emptyMap()); @@ -426,7 +435,7 @@ public final IgniteInternalFuture> loadAsync( expiryPlc = expiryPolicy(null); // Optimization: try to resolve value locally and escape 'get future' creation. - if (!forcePrimary && ctx.affinityNode()) { + if (!forcePrimary && ctx.affinityNode() && (!ctx.mvccEnabled() || mvccSnapshot != null)) { try { Map locVals = null; @@ -499,6 +508,7 @@ public final IgniteInternalFuture> loadAsync( taskName, expiryPlc, !deserializeBinary, + mvccSnapshot, null); if (getRes != null) { @@ -517,7 +527,8 @@ public final IgniteInternalFuture> loadAsync( null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + mvccSnapshot); } // Entry was not in memory or in swap, so we remove it from cache. @@ -600,7 +611,8 @@ else if (!skipVals && ctx.statisticsEnabled()) expiryPlc, skipVals, needVer, - keepCacheObj); + keepCacheObj, + mvccSnapshot); fut.init(topVer); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java index fe216a00379ac..52691f02521b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java @@ -42,6 +42,9 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -537,6 +540,10 @@ void onResult(GridDhtForceKeysResponse res) { if (entry.initialValue( info.value(), info.version(), + cctx.mvccEnabled() ? ((MvccVersionAware)info).mvccVersion() : null, + cctx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccVersion() : null, + cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccTxState() : TxState.NA, + cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccTxState() : TxState.NA, info.ttl(), info.expireTime(), true, 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 5b5f3444027e3..a0f82eab217d0 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 @@ -52,6 +52,9 @@ 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.GridDhtPartitionTopology; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -881,6 +884,10 @@ private boolean preloadEntry( if (cached.initialValue( entry.value(), entry.version(), + cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccVersion() : null, + cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccVersion() : null, + cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccTxState() : TxState.NA, + cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccTxState() : TxState.NA, entry.ttl(), entry.expireTime(), true, 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 ea7f4c9b4aacd..7ce4f7e7db496 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 @@ -33,10 +33,15 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; import org.apache.ignite.internal.processors.cache.IgniteRebalanceIterator; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; 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.mvcc.MvccUpdateVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.T3; @@ -361,14 +366,39 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (!remainingParts.contains(part)) continue; - GridCacheEntryInfo info = new GridCacheEntryInfo(); + GridCacheEntryInfo info = grp.mvccEnabled() ? + new GridCacheMvccEntryInfo() : new GridCacheEntryInfo(); info.key(row.key()); - info.expireTime(row.expireTime()); - info.version(row.version()); - info.value(row.value()); info.cacheId(row.cacheId()); + if (grp.mvccEnabled()) { + byte txState = row.mvccTxState() != TxState.NA ? row.mvccTxState() : + MvccUtils.state(grp, row.mvccCoordinatorVersion(), row.mvccCounter(), + row.mvccOperationCounter()); + + if (txState != TxState.COMMITTED) + continue; + + ((MvccVersionAware)info).mvccVersion(row); + ((GridCacheMvccEntryInfo)info).mvccTxState(TxState.COMMITTED); + + byte newTxState = row.newMvccTxState() != TxState.NA ? row.newMvccTxState() : + MvccUtils.state(grp, row.newMvccCoordinatorVersion(), row.newMvccCounter(), + row.newMvccOperationCounter()); + + if (newTxState != TxState.ABORTED) { + ((MvccUpdateVersionAware)info).newMvccVersion(row); + + if (newTxState == TxState.COMMITTED) + ((GridCacheMvccEntryInfo)info).newMvccTxState(TxState.COMMITTED); + } + } + + info.value(row.value()); + info.version(row.version()); + info.expireTime(row.expireTime()); + if (preloadPred == null || preloadPred.apply(info)) s.addEntry0(part, iter.historical(part), info, grp.shared(), grp.cacheObjectContext()); else { 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 516dfb93f0144..357f3d78d0c06 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 @@ -90,6 +90,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsStateValidator; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.Latch; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -97,6 +98,7 @@ import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage; import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -647,7 +649,17 @@ public void init(boolean newCrd) throws IgniteInterruptedCheckedException { boolean crdNode = crd != null && crd.isLocal(); - exchCtx = new ExchangeContext(crdNode, this); + MvccCoordinator mvccCrd = firstEvtDiscoCache.mvccCoordinator(); + + boolean mvccCrdChange = mvccCrd != null && + (initialVersion().equals(mvccCrd.topologyVersion()) || activateCluster()); + + // Mvcc coordinator should has been initialized before exchange context is created. + cctx.kernalContext().coordinators().updateCoordinator(mvccCrd); + + exchCtx = new ExchangeContext(crdNode, mvccCrdChange, this); + + cctx.kernalContext().coordinators().onExchangeStart(mvccCrd, exchCtx, crd); assert state == null : state; @@ -658,6 +670,8 @@ public void init(boolean newCrd) throws IgniteInterruptedCheckedException { if (exchLog.isInfoEnabled()) { exchLog.info("Started exchange init [topVer=" + topVer + + ", mvccCrd=" + mvccCrd + + ", mvccCrdChange=" + mvccCrdChange + ", crd=" + crdNode + ", evt=" + IgniteUtils.gridEventName(firstDiscoEvt.type()) + ", evtNode=" + firstDiscoEvt.eventNode().id() + @@ -745,7 +759,7 @@ else if (msg instanceof WalStateAbstractMessage) } } - updateTopologies(crdNode); + updateTopologies(crd, crdNode, cctx.coordinators().currentCoordinator()); switch (exchange) { case ALL: { @@ -895,10 +909,12 @@ private void initTopologies() throws IgniteCheckedException { /** * Updates topology versions and discovery caches on all topologies. * + * @param exchCrd Exchange coordinator node. * @param crd Coordinator flag. + * @param mvccCrd Mvcc coordinator. * @throws IgniteCheckedException If failed. */ - private void updateTopologies(boolean crd) throws IgniteCheckedException { + private void updateTopologies(ClusterNode exchCrd, boolean crd, MvccCoordinator mvccCrd) throws IgniteCheckedException { for (CacheGroupContext grp : cctx.cache().cacheGroups()) { if (grp.isLocal()) continue; @@ -925,12 +941,18 @@ private void updateTopologies(boolean crd) throws IgniteCheckedException { top.updateTopologyVersion( this, events().discoveryCache(), + mvccCrd, updSeq, cacheGroupStopping(grp.groupId())); } - for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) - top.updateTopologyVersion(this, events().discoveryCache(), -1, cacheGroupStopping(top.groupId())); + for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) { + top.updateTopologyVersion(this, + events().discoveryCache(), + mvccCrd, + -1, + cacheGroupStopping(top.groupId())); + } } /** @@ -1570,6 +1592,12 @@ private void sendLocalPartitions(ClusterNode node) throws IgniteCheckedException msg.partitionHistoryCounters(partHistReserved0); } + if (exchCtx.newMvccCoordinator() && cctx.coordinators().currentCoordinatorId().equals(node.id())) { + Map activeQueries = exchCtx.activeQueries(); + + msg.activeQueries(activeQueries != null ? activeQueries.get(cctx.localNodeId()) : null); + } + if ((stateChangeExchange() || dynamicCacheStartExchange()) && exchangeLocE != null) msg.setError(exchangeLocE); else if (localJoinExchange()) @@ -1813,6 +1841,10 @@ public void finishMerged() { if (!cctx.localNode().isClient()) tryToPerformLocalSnapshotOperation(); + if (err == null) + cctx.coordinators().onExchangeDone(exchCtx.newMvccCoordinator(), exchCtx.events().discoveryCache(), + exchCtx.activeQueries()); + cctx.cache().onExchangeDone(initialVersion(), exchActions, err); cctx.exchange().onExchangeDone(res, initialVersion(), err); @@ -2319,6 +2351,9 @@ public void waitAndReplyToNode(final UUID nodeId, final GridDhtPartitionsSingleM */ private void processSingleMessage(UUID nodeId, GridDhtPartitionsSingleMessage msg) { if (msg.client()) { + if (msg.activeQueries() != null) + cctx.coordinators().processClientActiveQueries(nodeId, msg.activeQueries()); + waitAndReplyToNode(nodeId, msg); return; @@ -2856,6 +2891,9 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe for (Map.Entry e : msgs.entrySet()) { GridDhtPartitionsSingleMessage msg = e.getValue(); + if (exchCtx.newMvccCoordinator()) + exchCtx.addActiveQueries(e.getKey(), msg.activeQueries()); + // Apply update counters after all single messages are received. for (Map.Entry entry : msg.partitions().entrySet()) { Integer grpId = entry.getKey(); 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 804cc030489f7..77d7eabfc780e 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 @@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; @@ -108,6 +109,9 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes */ private GridDhtPartitionsFullMessage finishMsg; + /** */ + private GridLongList activeQryTrackers; + /** * Required by {@link Externalizable}. */ @@ -131,6 +135,20 @@ public GridDhtPartitionsSingleMessage(GridDhtPartitionExchangeId exchId, this.compress = compress; } + /** + * @return Active queries started with previous coordinator. + */ + GridLongList activeQueries() { + return activeQryTrackers; + } + + /** + * @param activeQrys Active queries started with previous coordinator. + */ + void activeQueries(GridLongList activeQrys) { + this.activeQryTrackers = activeQrys; + } + /** * @param finishMsg Exchange finish message (used to restore exchange state on new coordinator). */ @@ -331,7 +349,7 @@ public void setError(Exception ex) { byte[] partsBytes0 = null; byte[] partCntrsBytes0 = null; byte[] partHistCntrsBytes0 = null; - byte[] partSizesBytes0 = null; + byte[] partsSizesBytes0 = null; byte[] errBytes0 = null; if (parts != null && partsBytes == null) @@ -344,7 +362,7 @@ public void setError(Exception ex) { partHistCntrsBytes0 = U.marshal(ctx, partHistCntrs); if (partsSizes != null && partsSizesBytes == null) - partSizesBytes0 = U.marshal(ctx, partsSizes); + partsSizesBytes0 = U.marshal(ctx, partsSizes); if (err != null && errBytes == null) errBytes0 = U.marshal(ctx, err); @@ -356,13 +374,13 @@ public void setError(Exception ex) { byte[] partsBytesZip = U.zip(partsBytes0); byte[] partCntrsBytesZip = U.zip(partCntrsBytes0); byte[] partHistCntrsBytesZip = U.zip(partHistCntrsBytes0); - byte[] partSizesBytesZip = U.zip(partSizesBytes0); + byte[] partsSizesBytesZip = U.zip(partsSizesBytes0); byte[] exBytesZip = U.zip(errBytes0); partsBytes0 = partsBytesZip; partCntrsBytes0 = partCntrsBytesZip; partHistCntrsBytes0 = partHistCntrsBytesZip; - partSizesBytes0 = partSizesBytesZip; + partsSizesBytes0 = partsSizesBytesZip; errBytes0 = exBytesZip; compressed(true); @@ -375,7 +393,7 @@ public void setError(Exception ex) { partsBytes = partsBytes0; partCntrsBytes = partCntrsBytes0; partHistCntrsBytes = partHistCntrsBytes0; - partsSizesBytes = partSizesBytes0; + partsSizesBytes = partsSizesBytes0; errBytes = errBytes0; } } @@ -456,54 +474,60 @@ public void setError(Exception ex) { switch (writer.state()) { case 5: - if (!writer.writeBoolean("client", client)) + if (!writer.writeMessage("activeQryTrackers", activeQryTrackers)) return false; writer.incrementState(); case 6: - if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT)) + if (!writer.writeBoolean("client", client)) return false; writer.incrementState(); case 7: - if (!writer.writeByteArray("errBytes", errBytes)) + if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT)) return false; writer.incrementState(); case 8: - if (!writer.writeMessage("finishMsg", finishMsg)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; writer.incrementState(); case 9: - if (!writer.writeCollection("grpsAffRequest", grpsAffRequest, MessageCollectionItemType.INT)) + if (!writer.writeMessage("finishMsg", finishMsg)) return false; writer.incrementState(); case 10: - if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) + if (!writer.writeCollection("grpsAffRequest", grpsAffRequest, MessageCollectionItemType.INT)) return false; writer.incrementState(); case 11: - if (!writer.writeByteArray("partHistCntrsBytes", partHistCntrsBytes)) + if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) return false; writer.incrementState(); case 12: - if (!writer.writeByteArray("partsBytes", partsBytes)) + if (!writer.writeByteArray("partHistCntrsBytes", partHistCntrsBytes)) return false; writer.incrementState(); case 13: + if (!writer.writeByteArray("partsBytes", partsBytes)) + return false; + + writer.incrementState(); + + case 14: if (!writer.writeByteArray("partsSizesBytes", partsSizesBytes)) return false; @@ -525,7 +549,7 @@ public void setError(Exception ex) { switch (reader.state()) { case 5: - client = reader.readBoolean("client"); + activeQryTrackers = reader.readMessage("activeQryTrackers"); if (!reader.isLastRead()) return false; @@ -533,7 +557,7 @@ public void setError(Exception ex) { reader.incrementState(); case 6: - dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false); + client = reader.readBoolean("client"); if (!reader.isLastRead()) return false; @@ -541,7 +565,7 @@ public void setError(Exception ex) { reader.incrementState(); case 7: - errBytes = reader.readByteArray("errBytes"); + dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false); if (!reader.isLastRead()) return false; @@ -549,7 +573,7 @@ public void setError(Exception ex) { reader.incrementState(); case 8: - finishMsg = reader.readMessage("finishMsg"); + errBytes = reader.readByteArray("errBytes"); if (!reader.isLastRead()) return false; @@ -557,7 +581,7 @@ public void setError(Exception ex) { reader.incrementState(); case 9: - grpsAffRequest = reader.readCollection("grpsAffRequest", MessageCollectionItemType.INT); + finishMsg = reader.readMessage("finishMsg"); if (!reader.isLastRead()) return false; @@ -565,7 +589,7 @@ public void setError(Exception ex) { reader.incrementState(); case 10: - partCntrsBytes = reader.readByteArray("partCntrsBytes"); + grpsAffRequest = reader.readCollection("grpsAffRequest", MessageCollectionItemType.INT); if (!reader.isLastRead()) return false; @@ -573,7 +597,7 @@ public void setError(Exception ex) { reader.incrementState(); case 11: - partHistCntrsBytes = reader.readByteArray("partHistCntrsBytes"); + partCntrsBytes = reader.readByteArray("partCntrsBytes"); if (!reader.isLastRead()) return false; @@ -581,7 +605,7 @@ public void setError(Exception ex) { reader.incrementState(); case 12: - partsBytes = reader.readByteArray("partsBytes"); + partHistCntrsBytes = reader.readByteArray("partHistCntrsBytes"); if (!reader.isLastRead()) return false; @@ -589,6 +613,14 @@ public void setError(Exception ex) { reader.incrementState(); case 13: + partsBytes = reader.readByteArray("partsBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 14: partsSizesBytes = reader.readByteArray("partsSizesBytes"); if (!reader.isLastRead()) @@ -607,7 +639,7 @@ public void setError(Exception ex) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 14; + return 15; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/AckCoordinatorOnRollback.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/AckCoordinatorOnRollback.java new file mode 100644 index 0000000000000..1648da926c555 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/AckCoordinatorOnRollback.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.near; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.util.typedef.CIX1; + +/** */ +public class AckCoordinatorOnRollback extends CIX1> { + /** */ + private static final long serialVersionUID = 8172699207968328284L; + + /** */ + private final GridNearTxLocal tx; + + /** + * @param tx Transaction. + */ + public AckCoordinatorOnRollback(GridNearTxLocal tx) { + this.tx = tx; + } + + /** {@inheritDoc} */ + @Override public void applyx(IgniteInternalFuture fut) throws IgniteCheckedException { + assert fut.isDone(); + + MvccQueryTracker tracker = tx.mvccQueryTracker(); + MvccSnapshot mvccSnapshot = tx.mvccSnapshot(); + + if (tracker != null) // Optimistic tx. + tracker.onDone(tx, false); + else if (mvccSnapshot != null)// Pessimistic tx. + tx.context().coordinators().ackTxRollback(mvccSnapshot); + } +} 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 ee0651814b306..7cf5a6b31940a 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 @@ -319,7 +319,8 @@ private void map( taskName == null ? 0 : taskName.hashCode(), expiryPlc, skipVals, - recovery); + recovery, + null); // TODO IGNITE-7371 final Collection invalidParts = fut.invalidPartitions(); @@ -382,7 +383,8 @@ private void map( true, skipVals, cctx.deploymentEnabled(), - recovery); + recovery, + null); // TODO IGNITE-7371 add(fut); // Append new future. @@ -455,7 +457,8 @@ private Map map( taskName, expiryPlc, !deserializeBinary, - null); + null, + null); // TODO IGNITE-7371 if (res != null) { v = res.value(); @@ -473,7 +476,8 @@ private Map map( null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + null); // TODO IGNITE-7371 } } @@ -599,7 +603,8 @@ private boolean localDhtGet(KeyCacheObject key, taskName, expiryPlc, !deserializeBinary, - null); + null, + null); // TODO IGNITE-7371 if (res != null) { v = res.value(); @@ -617,7 +622,8 @@ private boolean localDhtGet(KeyCacheObject key, null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + null); // TODO IGNITE-7371 } // Entry was not in memory or in swap, so we remove it from cache. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java index dcb167d81a892..c891bfb7dc3c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionable; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -45,6 +46,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * Get request. Responsible for obtaining entry from primary node. 'Near' means 'Initiating node' here, not 'Near Cache'. @@ -106,6 +108,9 @@ public class GridNearGetRequest extends GridCacheIdMessage implements GridCacheD /** TTL for read operation. */ private long accessTtl; + /** */ + private MvccSnapshot mvccSnapshot; + /** * Empty constructor required for {@link Externalizable}. */ @@ -128,6 +133,7 @@ public GridNearGetRequest() { * @param createTtl New TTL to set after entry is created, -1 to leave unchanged. * @param accessTtl New TTL to set after entry is accessed, -1 to leave unchanged. * @param addDepInfo Deployment info. + * @param mvccSnapshot Mvcc snapshot. */ public GridNearGetRequest( int cacheId, @@ -144,7 +150,8 @@ public GridNearGetRequest( boolean addReader, boolean skipVals, boolean addDepInfo, - boolean recovery + boolean recovery, + @Nullable MvccSnapshot mvccSnapshot ) { assert futId != null; assert miniId != null; @@ -173,6 +180,7 @@ public GridNearGetRequest( this.createTtl = createTtl; this.accessTtl = accessTtl; this.addDepInfo = addDepInfo; + this.mvccSnapshot = mvccSnapshot; if (readThrough) flags |= READ_THROUGH_FLAG_MASK; @@ -187,6 +195,13 @@ public GridNearGetRequest( flags |= RECOVERY_FLAG_MASK; } + /** + * @return Mvcc version. + */ + @Nullable public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + /** * @return Future ID. */ @@ -382,30 +397,36 @@ public long accessTtl() { writer.incrementState(); case 9: - if (!writer.writeCollection("readersFlags", readersFlags, MessageCollectionItemType.BOOLEAN)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 10: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeCollection("readersFlags", readersFlags, MessageCollectionItemType.BOOLEAN)) return false; writer.incrementState(); case 11: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 12: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; writer.incrementState(); case 13: + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + case 14: if (!writer.writeMessage("ver", ver)) return false; @@ -476,7 +497,7 @@ public long accessTtl() { reader.incrementState(); case 9: - readersFlags = reader.readCollection("readersFlags", MessageCollectionItemType.BOOLEAN); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -484,7 +505,7 @@ public long accessTtl() { reader.incrementState(); case 10: - subjId = reader.readUuid("subjId"); + readersFlags = reader.readCollection("readersFlags", MessageCollectionItemType.BOOLEAN); if (!reader.isLastRead()) return false; @@ -492,7 +513,7 @@ public long accessTtl() { reader.incrementState(); case 11: - taskNameHash = reader.readInt("taskNameHash"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -500,7 +521,7 @@ public long accessTtl() { reader.incrementState(); case 12: - topVer = reader.readMessage("topVer"); + taskNameHash = reader.readInt("taskNameHash"); if (!reader.isLastRead()) return false; @@ -508,6 +529,14 @@ public long accessTtl() { reader.incrementState(); case 13: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 14: ver = reader.readMessage("ver"); if (!reader.isLastRead()) @@ -527,7 +556,7 @@ public long accessTtl() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 14; + return 15; } /** {@inheritDoc} */ 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 a997c45ec4395..140c1d5fdbb7d 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 @@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -50,7 +51,6 @@ import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.P1; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -59,6 +59,7 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.noCoordinatorError; import static org.apache.ignite.transactions.TransactionState.PREPARED; import static org.apache.ignite.transactions.TransactionState.PREPARING; @@ -184,10 +185,20 @@ private void onError(@Nullable GridDistributedTxMapping m, Throwable e) { tx.removeMapping(m.primary().id()); } + prepareError(e); + } + + /** + * @param e Error. + */ + private void prepareError(Throwable e) { ERR_UPD.compareAndSet(this, null, e); if (keyLockFut != null) keyLockFut.onDone(e); + + if (mvccVerFut != null) + mvccVerFut.onDone(); } /** {@inheritDoc} */ @@ -229,7 +240,7 @@ private MiniFuture miniFuture(int miniId) { // Avoid iterator creation. for (int i = 0; i < size; i++) { - IgniteInternalFuture fut = future(i); + IgniteInternalFuture fut = future(i); if (!isMini(fut)) continue; @@ -338,11 +349,25 @@ private void prepare( boolean hasNearCache = false; + MvccCoordinator mvccCrd = null; + for (IgniteTxEntry write : writes) { map(write, topVer, mappings, txMapping, remap, topLocked); - if (write.context().isNear()) + GridCacheContext cctx = write.context(); + + if (cctx.isNear()) hasNearCache = true; + + if (cctx.mvccEnabled() && mvccCrd == null) { + mvccCrd = cctx.affinity().mvccCoordinator(topVer); + + if (mvccCrd == null) { + onDone(noCoordinatorError(topVer)); + + return; + } + } } for (IgniteTxEntry read : reads) @@ -358,6 +383,8 @@ private void prepare( return; } + assert !tx.txState().mvccEnabled(cctx) || mvccCrd != null || F.isEmpty(writes); + tx.addEntryMapping(mappings.values()); cctx.mvcc().recheckPendingLocks(); @@ -369,23 +396,32 @@ private void prepare( MiniFuture locNearEntriesFut = null; + int lockCnt = keyLockFut != null ? 1 : 0; + // Create futures in advance to have all futures when process {@link GridNearTxPrepareResponse#clientRemapVersion}. for (GridDistributedTxMapping m : mappings.values()) { assert !m.empty(); MiniFuture fut = new MiniFuture(this, m, ++miniId); - add(fut); + lockCnt++; + + add((IgniteInternalFuture)fut); if (m.primary().isLocal() && m.hasNearCacheEntries() && m.hasColocatedCacheEntries()) { assert locNearEntriesFut == null; locNearEntriesFut = fut; - add(new MiniFuture(this, m, ++miniId)); + add((IgniteInternalFuture)new MiniFuture(this, m, ++miniId)); + + lockCnt++; } } + if (mvccCrd != null) + initMvccVersionFuture(lockCnt, remap); + Collection> futs = (Collection)futures(); Iterator> it = futs.iterator(); @@ -639,7 +675,7 @@ else if (!cacheCtx.isLocal()) if (keyLockFut == null) { keyLockFut = new KeyLockFuture(); - add(keyLockFut); + add((IgniteInternalFuture)keyLockFut); } keyLockFut.addLockKey(entry.txKey()); @@ -696,20 +732,20 @@ else if (cacheCtx.isColocated()) Collection futs = F.viewReadOnly(futures(), new C1, String>() { @Override public String apply(IgniteInternalFuture f) { - return "[node=" + ((MiniFuture)f).primary().id() + - ", loc=" + ((MiniFuture)f).primary().isLocal() + - ", done=" + f.isDone() + "]"; - } - }, - new P1>() { - @Override public boolean apply(IgniteInternalFuture f) { - return isMini(f); + if (isMini(f)) { + return "[node=" + ((MiniFuture)f).primary().id() + + ", loc=" + ((MiniFuture)f).primary().isLocal() + + ", done=" + f.isDone() + + ", err=" + f.error() + "]"; + } + else + return f.toString(); } }); return S.toString(GridNearOptimisticSerializableTxPrepareFuture.class, this, "innerFuts", futs, - "keyLockFut", keyLockFut, + "remap", remapFut != null, "tx", tx, "super", super.toString()); } @@ -760,7 +796,7 @@ private static class ClientRemapFutureReducer implements IgniteReducer { /** Receive result flag updater. */ - private static AtomicIntegerFieldUpdater RCV_RES_UPD = + private static final AtomicIntegerFieldUpdater RCV_RES_UPD = AtomicIntegerFieldUpdater.newUpdater(MiniFuture.class, "rcvRes"); /** */ @@ -924,7 +960,7 @@ void onResult(final GridNearTxPrepareResponse res, boolean updateMapping) { err0.retryReadyFuture(affFut); - ERR_UPD.compareAndSet(parent, null, err0); + parent.prepareError(err0); onDone(err0); } @@ -935,7 +971,7 @@ void onResult(final GridNearTxPrepareResponse res, boolean updateMapping) { parent); } - ERR_UPD.compareAndSet(parent, null, e); + parent.prepareError(e); onDone(e); } @@ -950,6 +986,9 @@ void onResult(final GridNearTxPrepareResponse res, boolean updateMapping) { // Finish this mini future (need result only on client node). onDone(parent.cctx.kernalContext().clientNode() ? res : null); + + if (parent.mvccVerFut != null) + parent.mvccVerFut.onLockReceived(); } } } 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 566789c17d92c..06d7a8c2cdf00 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 @@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -68,6 +69,7 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.noCoordinatorError; import static org.apache.ignite.transactions.TransactionState.PREPARED; import static org.apache.ignite.transactions.TransactionState.PREPARING; @@ -220,7 +222,7 @@ public Set requestedKeys() { int size = futuresCountNoLock(); for (int i = 0; i < size; i++) { - IgniteInternalFuture fut = future(i); + IgniteInternalFuture fut = future(i); if (isMini(fut) && !fut.isDone()) { MiniFuture miniFut = (MiniFuture)fut; @@ -254,7 +256,7 @@ private MiniFuture miniFuture(int miniId) { // Avoid iterator creation. for (int i = size - 1; i >= 0; i--) { - IgniteInternalFuture fut = future(i); + IgniteInternalFuture fut = future(i); if (!isMini(fut)) continue; @@ -380,6 +382,18 @@ else if (write.context().isColocated()) tx.colocatedLocallyMapped(true); } + if (write.context().mvccEnabled()) { + MvccCoordinator mvccCrd = write.context().affinity().mvccCoordinator(topVer); + + if (mvccCrd == null) { + onDone(noCoordinatorError(topVer)); + + return; + } + + initMvccVersionFuture(keyLockFut != null ? 2 : 1, remap); + } + if (keyLockFut != null) keyLockFut.onAllKeysAdded(); @@ -424,6 +438,8 @@ private void prepare( boolean hasNearCache = false; + MvccCoordinator mvccCrd = null; + for (IgniteTxEntry write : writes) { write.clearEntryReadVersion(); @@ -433,6 +449,16 @@ private void prepare( // an exception occurred while transaction mapping, stop further processing break; + if (write.context().mvccEnabled() && mvccCrd == null) { + mvccCrd = write.context().affinity().mvccCoordinator(topVer); + + if (mvccCrd == null) { + onDone(noCoordinatorError(topVer)); + + break; + } + } + if (write.context().isNear()) hasNearCache = true; @@ -472,6 +498,11 @@ else if (write.context().isColocated()) return; } + assert !tx.txState().mvccEnabled(cctx) || mvccCrd != null; + + if (mvccCrd != null) + initMvccVersionFuture(keyLockFut != null ? 2 : 1, remap); + if (keyLockFut != null) keyLockFut.onAllKeysAdded(); @@ -495,8 +526,12 @@ else if (write.context().isColocated()) private void proceedPrepare(final Queue mappings) { final GridDistributedTxMapping m = mappings.poll(); - if (m == null) + if (m == null) { + if (mvccVerFut != null) + mvccVerFut.onLockReceived(); + return; + } proceedPrepare(m, mappings); } @@ -562,7 +597,7 @@ private void proceedPrepare(GridDistributedTxMapping m, @Nullable final Queue fut = future(i); + IgniteInternalFuture fut = future(i); if (isMini(fut) && !fut.isDone()) { MiniFuture miniFut = (MiniFuture)fut; @@ -756,7 +791,7 @@ private void onTimeout() { } } - add(new GridEmbeddedFuture<>(new IgniteBiClosure() { + add(new GridEmbeddedFuture<>(new IgniteBiClosure() { @Override public GridNearTxPrepareResponse apply(TxDeadlock deadlock, Exception e) { if (e != null) U.warn(log, "Failed to detect deadlock.", e); @@ -787,7 +822,7 @@ private void onTimeout() { /** {@inheritDoc} */ @Override public void addDiagnosticRequest(IgniteDiagnosticPrepareContext ctx) { if (!isDone()) { - for (IgniteInternalFuture fut : futures()) { + for (IgniteInternalFuture fut : futures()) { if (!fut.isDone()) { if (fut instanceof MiniFuture) { MiniFuture miniFut = (MiniFuture)fut; @@ -843,19 +878,22 @@ else if (fut instanceof KeyLockFuture) { @Override public String toString() { Collection futs = F.viewReadOnly(futures(), new C1, String>() { @Override public String apply(IgniteInternalFuture f) { - return "[node=" + ((MiniFuture)f).node().id() + - ", loc=" + ((MiniFuture)f).node().isLocal() + - ", done=" + f.isDone() + "]"; + if (isMini(f)) { + return "[node=" + ((MiniFuture)f).node().id() + + ", loc=" + ((MiniFuture)f).node().isLocal() + + ", done=" + f.isDone() + "]"; + } + else + return f.toString(); } - }, new P1>() { - @Override public boolean apply(IgniteInternalFuture fut) { + }, new P1>() { + @Override public boolean apply(IgniteInternalFuture fut) { return isMini(fut); } }); return S.toString(GridNearOptimisticTxPrepareFuture.class, this, "innerFuts", futs, - "keyLockFut", keyLockFut, "tx", tx, "super", super.toString()); } @@ -1001,6 +1039,8 @@ void onResult(final GridNearTxPrepareResponse res) { // Proceed prepare before finishing mini future. if (mappings != null) parent.proceedPrepare(mappings); + else if (parent.mvccVerFut != null) + parent.mvccVerFut.onLockReceived(); // Finish this mini future. onDone((GridNearTxPrepareResponse)null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java index 45f89416b8776..6f541d330b5a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java @@ -18,19 +18,20 @@ package org.apache.ignite.internal.processors.cache.distributed.near; import java.util.Collection; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotResponseListener; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; -import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.lang.GridAbsClosureX; 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; @@ -41,10 +42,21 @@ * */ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearTxPrepareFutureAdapter { + /** */ + private static final long serialVersionUID = 7460376140787916619L; + + /** */ + private static final AtomicIntegerFieldUpdater LOCK_CNT_UPD = + AtomicIntegerFieldUpdater.newUpdater(MvccSnapshotFutureExt.class, "lockCnt"); + /** */ @GridToStringExclude protected KeyLockFuture keyLockFut; + /** */ + @GridToStringExclude + protected MvccSnapshotFutureExt mvccVerFut; + /** * @param cctx Context. * @param tx Transaction. @@ -73,7 +85,7 @@ protected GridNearOptimisticTxPrepareFutureAdapter(GridCacheSharedContext cctx, } if (keyLockFut != null) - add(keyLockFut); + add((IgniteInternalFuture)keyLockFut); } } @@ -201,6 +213,29 @@ protected final void prepareOnTopology(final boolean remap, @Nullable final Runn */ protected abstract void prepare0(boolean remap, boolean topLocked); + /** + * @param lockCnt Expected number of lock responses. + * @param remap Remap flag. + */ + @SuppressWarnings("unchecked") + final void initMvccVersionFuture(int lockCnt, boolean remap) { + if (!remap) { + mvccVerFut = new MvccSnapshotFutureExt(); + + mvccVerFut.init(lockCnt); + + if (keyLockFut != null) + keyLockFut.listen(mvccVerFut); + + add((IgniteInternalFuture)mvccVerFut); + } + else { + assert mvccVerFut != null; + + mvccVerFut.init(lockCnt); + } + } + /** * @param e Exception. * @param timedOut {@code True} if timed out. @@ -223,7 +258,7 @@ protected boolean errorOrTimeoutOnTopologyVersion(IgniteCheckedException e, bool /** * Keys lock future. */ - protected static class KeyLockFuture extends GridFutureAdapter { + protected static class KeyLockFuture extends GridFutureAdapter { /** */ @GridToStringInclude protected Collection lockKeys = new GridConcurrentHashSet<>(); @@ -258,24 +293,20 @@ protected void onAllKeysAdded() { checkLocks(); } - /** - * @return {@code True} if all locks are owned. - */ - private boolean checkLocks() { + /** */ + private void checkLocks() { boolean locked = lockKeys.isEmpty(); if (locked && allKeysAdded) { if (log.isDebugEnabled()) log.debug("All locks are acquired for near prepare future: " + this); - onDone((GridNearTxPrepareResponse)null); + onDone((Void)null); } else { if (log.isDebugEnabled()) log.debug("Still waiting for locks [fut=" + this + ", keys=" + lockKeys + ']'); } - - return locked; } /** {@inheritDoc} */ @@ -283,4 +314,82 @@ private boolean checkLocks() { return S.toString(KeyLockFuture.class, this, super.toString()); } } + + /** + * + */ + class MvccSnapshotFutureExt extends GridFutureAdapter implements MvccSnapshotResponseListener, IgniteInClosure> { + /** */ + private static final long serialVersionUID = 5883078648683911226L; + + /** */ + volatile int lockCnt; + + /** {@inheritDoc} */ + @Override public void apply(IgniteInternalFuture keyLockFut) { + try { + keyLockFut.get(); + + onLockReceived(); + } + catch (IgniteCheckedException e) { + if (log.isDebugEnabled()) + log.debug("MvccSnapshotFutureExt ignores key lock future failure: " + e); + } + } + + /** + * @param lockCnt Expected number of lock responses. + */ + void init(int lockCnt) { + assert lockCnt > 0; + + this.lockCnt = lockCnt; + + assert !isDone(); + } + + /** */ + void onLockReceived() { + int remaining = LOCK_CNT_UPD.decrementAndGet(this); + + assert remaining >= 0 : remaining; + + if (remaining == 0) { + try { + MvccSnapshot snapshot = cctx.coordinators().tryRequestSnapshotLocal(tx); + + if (snapshot != null) + onResponse(snapshot); + else + cctx.coordinators().requestSnapshotAsync(tx, this); + } + catch (ClusterTopologyCheckedException e) { + onError(e); + } + } + } + + /** {@inheritDoc} */ + @Override public void onResponse(MvccSnapshot res) { + tx.mvccSnapshot(res); + + onDone(); + } + + /** {@inheritDoc} */ + @Override public void onError(IgniteCheckedException e) { + if (e instanceof ClusterTopologyCheckedException) + ((ClusterTopologyCheckedException)e).retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); + + ERR_UPD.compareAndSet(GridNearOptimisticTxPrepareFutureAdapter.this, null, e); + + onDone(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccSnapshotFutureExt.class, this, super.toString()); + } + } } 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 ffde0f39b0236..dbf54c23af3e5 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 @@ -36,6 +36,9 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotResponseListener; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; @@ -43,11 +46,13 @@ import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; +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; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.noCoordinatorError; import static org.apache.ignite.transactions.TransactionState.PREPARED; import static org.apache.ignite.transactions.TransactionState.PREPARING; @@ -55,6 +60,9 @@ * */ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureAdapter { + /** */ + private static final long serialVersionUID = 4014479758215810181L; + /** * @param cctx Context. * @param tx Transaction. @@ -80,17 +88,19 @@ public GridNearPessimisticTxPrepareFuture(GridCacheSharedContext cctx, GridNearT boolean found = false; for (IgniteInternalFuture fut : futures()) { - MiniFuture f = (MiniFuture)fut; + if (fut instanceof MiniFuture) { + MiniFuture f = (MiniFuture)fut; - if (f.primary().id().equals(nodeId)) { - ClusterTopologyCheckedException e = new ClusterTopologyCheckedException("Remote node left grid: " + - nodeId); + if (f.primary().id().equals(nodeId)) { + ClusterTopologyCheckedException e = new ClusterTopologyCheckedException("Remote node left grid: " + + nodeId); - e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); + e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); - f.onNodeLeft(e); + f.onNodeLeft(e); - found = true; + found = true; + } } } @@ -142,13 +152,17 @@ private MiniFuture miniFuture(int miniId) { // Avoid iterator creation. for (int i = 0; i < size; i++) { - MiniFuture mini = (MiniFuture)future(i); + IgniteInternalFuture fut = future(i); - if (mini.futureId() == miniId) { - if (!mini.isDone()) - return mini; - else - return null; + if (fut instanceof MiniFuture) { + MiniFuture mini = (MiniFuture)fut; + + if (mini.futureId() == miniId) { + if (!mini.isDone()) + return mini; + else + return null; + } } } } @@ -217,6 +231,8 @@ private GridNearTxPrepareRequest createRequest(Map> txNod true, tx.activeCachesDeploymentEnabled()); + req.queryUpdate(m.queryUpdate()); + for (IgniteTxEntry txEntry : writes) { if (txEntry.op() == TRANSFORM) req.addDhtVersion(txEntry.txKey(), null); @@ -231,6 +247,7 @@ private GridNearTxPrepareRequest createRequest(Map> txNod * @param miniId Mini future ID. * @param nearEntries {@code True} if prepare near cache entries. */ + @SuppressWarnings("unchecked") private void prepareLocal(GridNearTxPrepareRequest req, GridDistributedTxMapping m, int miniId, @@ -239,7 +256,7 @@ private void prepareLocal(GridNearTxPrepareRequest req, req.miniId(fut.futureId()); - add(fut); + add((IgniteInternalFuture)fut); IgniteInternalFuture prepFut = nearEntries ? cctx.tm().txHandler().prepareNearTxLocal(req) : @@ -260,6 +277,7 @@ private void prepareLocal(GridNearTxPrepareRequest req, /** * */ + @SuppressWarnings("unchecked") private void preparePessimistic() { Map mappings = new HashMap<>(); @@ -267,48 +285,77 @@ private void preparePessimistic() { GridDhtTxMapping txMapping = new GridDhtTxMapping(); + boolean queryMapped = false; + + for (GridDistributedTxMapping m : F.view(tx.mappings().mappings(), CU.FILTER_QUERY_MAPPING)) { + GridDistributedTxMapping nodeMapping = mappings.get(m.primary().id()); + + if(nodeMapping == null) + mappings.put(m.primary().id(), m); + + txMapping.addMapping(F.asList(m.primary())); + + queryMapped = true; + } + + MvccCoordinator mvccCrd = null; + boolean hasNearCache = false; - for (IgniteTxEntry txEntry : tx.allEntries()) { - txEntry.clearEntryReadVersion(); + if (!queryMapped) { + for (IgniteTxEntry txEntry : tx.allEntries()) { + txEntry.clearEntryReadVersion(); - GridCacheContext cacheCtx = txEntry.context(); + GridCacheContext cacheCtx = txEntry.context(); - if (cacheCtx.isNear()) - hasNearCache = true; + if (cacheCtx.isNear()) + hasNearCache = true; - List nodes; + List nodes; - if (!cacheCtx.isLocal()) { - GridDhtPartitionTopology top = cacheCtx.topology(); + if (!cacheCtx.isLocal()) { + GridDhtPartitionTopology top = cacheCtx.topology(); - nodes = top.nodes(cacheCtx.affinity().partition(txEntry.key()), topVer); - } - else - nodes = cacheCtx.affinity().nodesByKey(txEntry.key(), topVer); + nodes = top.nodes(cacheCtx.affinity().partition(txEntry.key()), topVer); + } + else + nodes = cacheCtx.affinity().nodesByKey(txEntry.key(), topVer); - if (F.isEmpty(nodes)) { - onDone(new ClusterTopologyServerNotFoundException("Failed to map keys to nodes (partition " + - "is not mapped to any node) [key=" + txEntry.key() + - ", partition=" + cacheCtx.affinity().partition(txEntry.key()) + ", topVer=" + topVer + ']')); + if (tx.mvccSnapshot() == null && mvccCrd == null && cacheCtx.mvccEnabled()) { + mvccCrd = cacheCtx.affinity().mvccCoordinator(topVer); - return; - } + if (mvccCrd == null) { + onDone(noCoordinatorError(topVer)); + + return; + } + } + + if (F.isEmpty(nodes)) { + onDone(new ClusterTopologyServerNotFoundException("Failed to map keys to nodes (partition " + + "is not mapped to any node) [key=" + txEntry.key() + + ", partition=" + cacheCtx.affinity().partition(txEntry.key()) + ", topVer=" + topVer + ']')); + + return; + } - ClusterNode primary = nodes.get(0); + ClusterNode primary = nodes.get(0); - GridDistributedTxMapping nodeMapping = mappings.get(primary.id()); + GridDistributedTxMapping nodeMapping = mappings.get(primary.id()); - if (nodeMapping == null) - mappings.put(primary.id(), nodeMapping = new GridDistributedTxMapping(primary)); + if (nodeMapping == null) + mappings.put(primary.id(), nodeMapping = new GridDistributedTxMapping(primary)); - txEntry.nodeId(primary.id()); + txEntry.nodeId(primary.id()); - nodeMapping.add(txEntry); + nodeMapping.add(txEntry); - txMapping.addMapping(nodes); + txMapping.addMapping(nodes); + } } + assert !tx.txState().mvccEnabled(cctx) || tx.mvccSnapshot() != null || mvccCrd != null; + tx.transactionNodes(txMapping.transactionNodes()); if (!hasNearCache) @@ -329,6 +376,16 @@ private void preparePessimistic() { for (final GridDistributedTxMapping m : mappings.values()) { final ClusterNode primary = m.primary(); + boolean needCntr = false; + + if (mvccCrd != null) { + if (tx.onePhaseCommit() || mvccCrd.nodeId().equals(primary.id())) { + needCntr = true; + + mvccCrd = null; + } + } + if (primary.isLocal()) { if (m.hasNearCacheEntries() && m.hasColocatedCacheEntries()) { GridNearTxPrepareRequest nearReq = createRequest(txMapping.transactionNodes(), @@ -337,6 +394,8 @@ private void preparePessimistic() { m.nearEntriesReads(), m.nearEntriesWrites()); + nearReq.requestMvccCounter(needCntr); + prepareLocal(nearReq, m, ++miniId, true); GridNearTxPrepareRequest colocatedReq = createRequest(txNodes, @@ -350,6 +409,8 @@ private void preparePessimistic() { else { GridNearTxPrepareRequest req = createRequest(txNodes, m, timeout, m.reads(), m.writes()); + req.requestMvccCounter(needCntr); + prepareLocal(req, m, ++miniId, m.hasNearCacheEntries()); } } @@ -360,11 +421,13 @@ private void preparePessimistic() { m.reads(), m.writes()); + req.requestMvccCounter(needCntr); + final MiniFuture fut = new MiniFuture(m, ++miniId); req.miniId(fut.futureId()); - add(fut); + add((IgniteInternalFuture)fut); try { cctx.io().send(primary, req, tx.ioPolicy()); @@ -392,6 +455,16 @@ private void preparePessimistic() { } } + if (mvccCrd != null) { + assert !tx.onePhaseCommit(); + + MvccSnapshotFutureExt fut = new MvccSnapshotFutureExt(); + + cctx.coordinators().requestSnapshotAsync(tx, fut); + + add((IgniteInternalFuture)fut); + } + markInitialized(); } @@ -423,20 +496,52 @@ private void preparePessimistic() { @Override public String toString() { Collection futs = F.viewReadOnly(futures(), new C1, String>() { @Override public String apply(IgniteInternalFuture f) { - return "[node=" + ((MiniFuture)f).primary().id() + - ", loc=" + ((MiniFuture)f).primary().isLocal() + - ", done=" + f.isDone() + "]"; + if (f instanceof MiniFuture) { + return "[node=" + ((MiniFuture)f).primary().id() + + ", loc=" + ((MiniFuture)f).primary().isLocal() + + ", done=" + f.isDone() + "]"; + } + else + return f.toString(); } }); return S.toString(GridNearPessimisticTxPrepareFuture.class, this, "innerFuts", futs, + "txId", tx.nearXidVersion(), "super", super.toString()); } /** * */ + private class MvccSnapshotFutureExt extends GridFutureAdapter implements MvccSnapshotResponseListener { + /** {@inheritDoc} */ + @Override public void onResponse(MvccSnapshot res) { + tx.mvccSnapshot(res); + + onDone(); + } + + /** {@inheritDoc} */ + @Override public void onError(IgniteCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Error on tx prepare [fut=" + this + ", err=" + e + ", tx=" + tx + ']'); + + if (ERR_UPD.compareAndSet(GridNearPessimisticTxPrepareFuture.this, null, e)) + tx.setRollbackOnly(); + + onDone(e); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccSnapshotFutureExt.class, this, super.toString()); + } + } + + + /** */ private class MiniFuture extends GridFutureAdapter { /** */ private final int futId; @@ -475,6 +580,9 @@ void onResult(GridNearTxPrepareResponse res, boolean updateMapping) { if (res.error() != null) onError(res.error()); else { + if (res.mvccSnapshot() != null) + tx.mvccSnapshot(res.mvccSnapshot()); + onPrepareResponse(m, res, updateMapping); onDone(res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java index 00ff4bb60bf95..d1aa2e96b9496 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java @@ -26,11 +26,13 @@ import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * @@ -81,6 +83,9 @@ public class GridNearSingleGetRequest extends GridCacheIdMessage implements Grid /** TTL for read operation. */ private long accessTtl; + /** */ + private MvccSnapshot mvccSnapshot; + /** * Empty constructor required for {@link Message}. */ @@ -103,6 +108,7 @@ public GridNearSingleGetRequest() { * @param addReader Add reader flag. * @param needVer {@code True} if entry version is needed. * @param addDepInfo Deployment info. + * @param mvccSnapshot MVCC snapshot. */ public GridNearSingleGetRequest( int cacheId, @@ -118,7 +124,8 @@ public GridNearSingleGetRequest( boolean addReader, boolean needVer, boolean addDepInfo, - boolean recovery + boolean recovery, + MvccSnapshot mvccSnapshot ) { assert key != null; @@ -131,6 +138,7 @@ public GridNearSingleGetRequest( this.createTtl = createTtl; this.accessTtl = accessTtl; this.addDepInfo = addDepInfo; + this.mvccSnapshot = mvccSnapshot; if (readThrough) flags |= READ_THROUGH_FLAG_MASK; @@ -148,6 +156,13 @@ public GridNearSingleGetRequest( flags |= RECOVERY_FLAG_MASK; } + /** + * @return Mvcc version. + */ + @Nullable public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + /** * @return Key. */ @@ -322,7 +337,7 @@ public boolean recovery() { reader.incrementState(); case 8: - subjId = reader.readUuid("subjId"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -330,7 +345,7 @@ public boolean recovery() { reader.incrementState(); case 9: - taskNameHash = reader.readInt("taskNameHash"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -338,6 +353,14 @@ public boolean recovery() { reader.incrementState(); case 10: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) @@ -396,18 +419,24 @@ public boolean recovery() { writer.incrementState(); case 8: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 9: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 10: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 11: if (!writer.writeMessage("topVer", topVer)) return false; @@ -430,7 +459,7 @@ public boolean recovery() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 11; + return 12; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java new file mode 100644 index 0000000000000..f484bd6ec3452 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java @@ -0,0 +1,499 @@ +/* + * 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.near; + +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteFutureCancelledCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheStoppedException; +import org.apache.ignite.internal.processors.cache.GridCacheCompoundIdentityFuture; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; +import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; +import org.apache.ignite.internal.processors.cache.distributed.dht.CompoundLockFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.CI1; +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.IgniteInClosure; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public abstract class GridNearTxAbstractEnlistFuture extends GridCacheCompoundIdentityFuture implements + GridCacheVersionedFuture { + /** */ + private static final long serialVersionUID = -6069985059301497282L; + + /** Done field updater. */ + private static final AtomicIntegerFieldUpdater DONE_UPD = + AtomicIntegerFieldUpdater.newUpdater(GridNearTxAbstractEnlistFuture.class, "done"); + + /** Done field updater. */ + private static final AtomicReferenceFieldUpdater EX_UPD = + AtomicReferenceFieldUpdater.newUpdater(GridNearTxAbstractEnlistFuture.class, Throwable.class, "ex"); + + /** Cache context. */ + @GridToStringExclude + protected final GridCacheContext cctx; + + /** Transaction. */ + protected final GridNearTxLocal tx; + + /** */ + protected AffinityTopologyVersion topVer; + + /** MVCC snapshot. */ + protected MvccSnapshot mvccSnapshot; + + /** Logger. */ + @GridToStringExclude + protected final IgniteLogger log; + + /** */ + protected long timeout; + + /** Initiated thread id. */ + protected final long threadId; + + /** Mvcc future id. */ + protected final IgniteUuid futId; + + /** Lock version. */ + protected final GridCacheVersion lockVer; + + /** */ + @GridToStringExclude + private GridDhtTxAbstractEnlistFuture localEnlistFuture; + + /** */ + @SuppressWarnings("unused") + @GridToStringExclude + protected volatile Throwable ex; + + /** */ + @SuppressWarnings("unused") + @GridToStringExclude + private volatile int done; + + /** Timeout object. */ + @GridToStringExclude + protected LockTimeoutObject timeoutObj; + + /** + * @param cctx Cache context. + * @param tx Transaction. + * @param timeout Timeout. + */ + public GridNearTxAbstractEnlistFuture( + GridCacheContext cctx, GridNearTxLocal tx, long timeout) { + super(CU.longReducer()); + + assert cctx != null; + assert tx != null; + + this.cctx = cctx; + this.tx = tx; + this.timeout = timeout; + + threadId = tx.threadId(); + lockVer = tx.xidVersion(); + futId = IgniteUuid.randomUuid(); + + mvccSnapshot = tx.mvccSnapshot(); + + assert mvccSnapshot != null; + + log = cctx.logger(getClass()); + } + + /** + * + */ + public void init() { + if (timeout < 0) { + // Time is out. + onDone(timeoutException()); + + return; + } + else if (timeout > 0) + timeoutObj = new LockTimeoutObject(); + + while(true) { + IgniteInternalFuture fut = tx.lockFuture(); + + if (fut == GridDhtTxLocalAdapter.ROLLBACK_FUT) { + onDone(tx.timedOut() ? tx.timeoutException() : tx.rollbackException()); + + return; + } + else if (fut != null) { + // Wait for previous future. + assert fut instanceof GridNearTxAbstractEnlistFuture + || fut instanceof GridDhtTxAbstractEnlistFuture + || fut instanceof CompoundLockFuture + || fut instanceof GridNearTxSelectForUpdateFuture : fut; + + // Terminate this future if parent future is terminated by rollback. + if (!fut.isDone()) { + fut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + if (fut.error() != null) + onDone(fut.error()); + } + }); + } + else if (fut.error() != null) + onDone(fut.error()); + + break; + } + else if (tx.updateLockFuture(null, this)) + break; + } + + boolean added = cctx.mvcc().addFuture(this); + + assert added : this; + + if (isDone()) { + cctx.mvcc().removeFuture(futId); + + return; + } + + try { + tx.addActiveCache(cctx, false); + } + catch (IgniteCheckedException e) { + onDone(e); + + return; + } + + if (timeoutObj != null) + cctx.time().addTimeoutObject(timeoutObj); + + // Obtain the topology version to use. + long threadId = Thread.currentThread().getId(); + + AffinityTopologyVersion topVer = cctx.mvcc().lastExplicitLockTopologyVersion(threadId); + + // If there is another system transaction in progress, use it's topology version to prevent deadlock. + if (topVer == null && tx.system()) + topVer = cctx.tm().lockedTopologyVersion(threadId, tx); + + if (topVer != null) + tx.topologyVersion(topVer); + + if (topVer == null) + topVer = tx.topologyVersionSnapshot(); + + if (topVer != null) { + for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()) { + if (fut.exchangeDone() && fut.topologyVersion().equals(topVer)) { + Throwable err = fut.validateCache(cctx, false, false, null, null); + + if (err != null) { + onDone(err); + + return; + } + + break; + } + } + + if (this.topVer == null) + this.topVer = topVer; + + map(true); + + return; + } + + mapOnTopology(); + } + + /** + * @param node Primary node. + * @throws IgniteCheckedException if future is already completed. + */ + protected synchronized void updateMappings(ClusterNode node) throws IgniteCheckedException { + checkCompleted(); + + IgniteTxMappings m = tx.mappings(); + + GridDistributedTxMapping mapping = m.get(node.id()); + + if (mapping == null) + m.put(mapping = new GridDistributedTxMapping(node)); + + mapping.markQueryUpdate(); + + if (node.isLocal()) + tx.colocatedLocallyMapped(true); + } + + /** + * @param fut Local enlist future. + * @throws IgniteCheckedException if future is already completed. + */ + protected synchronized void updateLocalFuture(GridDhtTxAbstractEnlistFuture fut) throws IgniteCheckedException { + checkCompleted(); + + assert localEnlistFuture == null; + + localEnlistFuture = fut; + } + + /** + * @param fut Local enlist future. + * @throws IgniteCheckedException if future is already completed. + */ + protected synchronized void clearLocalFuture(GridDhtTxAbstractEnlistFuture fut) throws IgniteCheckedException { + checkCompleted(); + + if (localEnlistFuture == fut) + localEnlistFuture = null; + } + + /** + * @throws IgniteCheckedException if future is already completed. + */ + protected void checkCompleted() throws IgniteCheckedException { + if (isDone()) + throw new IgniteCheckedException("Future is done."); + } + + + + /** + */ + private void mapOnTopology() { + cctx.topology().readLock(); + + try { + if (cctx.topology().stopping()) { + onDone(new CacheStoppedException(cctx.name())); + + return; + } + + GridDhtTopologyFuture fut = cctx.topologyVersionFuture(); + + if (fut.isDone()) { + Throwable err = fut.validateCache(cctx, false, false, null, null); + + if (err != null) { + onDone(err); + + return; + } + + AffinityTopologyVersion topVer = fut.topologyVersion(); + + if (tx != null) + tx.topologyVersion(topVer); + + if (this.topVer == null) + this.topVer = topVer; + + map(false); + } + else { + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + fut.get(); + + mapOnTopology(); + } + catch (IgniteCheckedException e) { + onDone(e); + } + finally { + cctx.shared().txContextReset(); + } + } + }); + } + } + finally { + cctx.topology().readUnlock(); + } + } + + /** {@inheritDoc} */ + @Override protected boolean processFailure(Throwable err, IgniteInternalFuture fut) { + if (ex != null || !EX_UPD.compareAndSet(this, null, err)) + ex.addSuppressed(err); + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean onDone(@Nullable Long res, @Nullable Throwable err, boolean cancelled) { + if (!DONE_UPD.compareAndSet(this, 0, 1)) + return false; + + cctx.tm().txContext(tx); + + Throwable ex0 = ex; + + if (ex0 != null) { + if (err != null) + ex0.addSuppressed(err); + + err = ex0; + } + + if (!cancelled && err == null) + tx.clearLockFuture(this); + else + tx.setRollbackOnly(); + + synchronized (this) { + boolean done = super.onDone(res, err, cancelled); + + assert done; + + GridDhtTxAbstractEnlistFuture localFuture0 = localEnlistFuture; + + if (localFuture0 != null && (err != null || cancelled)) + localFuture0.onDone(cancelled ? new IgniteFutureCancelledCheckedException("Future was cancelled: " + localFuture0) : err); + + // Clean up. + cctx.mvcc().removeVersionedFuture(this); + + if (timeoutObj != null) + cctx.time().removeTimeoutObject(timeoutObj); + + return true; + } + } + + /** {@inheritDoc} */ + @Override protected void logError(IgniteLogger log, String msg, Throwable e) { + // no-op + } + + /** {@inheritDoc} */ + @Override protected void logDebug(IgniteLogger log, String msg) { + // no-op + } + + /** {@inheritDoc} */ + @Override public boolean trackable() { + return true; + } + + /** {@inheritDoc} */ + @Override public void markNotTrackable() { + // No-op. + } + + @Override public GridCacheVersion version() { + return lockVer; + } + + @Override public boolean onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate owner) { + return false; + } + + @Override public IgniteUuid futureId() { + return futId; + } + + /** + * Gets remaining allowed time. + * + * @return Remaining time. {@code 0} if timeout isn't specified. {@code -1} if time is out. + * @throws IgniteTxTimeoutCheckedException If timed out. + */ + protected long remainingTime() throws IgniteTxTimeoutCheckedException { + if (timeout <= 0) + return 0; + + long timeLeft = timeout - (U.currentTimeMillis() - startTime()); + + if (timeLeft <= 0) + throw timeoutException(); + + return timeLeft; + } + + /** + * @return Timeout exception. + */ + @NotNull protected IgniteTxTimeoutCheckedException timeoutException() { + return new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " + + "transaction [timeout=" + timeout + ", tx=" + tx + ']'); + } + + /** + * Start iterating the data rows and form batches. + * + * @param topLocked Whether topology was already locked. + */ + protected abstract void map(boolean topLocked); + + /** + * Lock request timeout object. + */ + private class LockTimeoutObject extends GridTimeoutObjectAdapter { + /** + * Default constructor. + */ + LockTimeoutObject() { + super(timeout); + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (log.isDebugEnabled()) + log.debug("Timed out waiting for lock response: " + this); + + onDone(timeoutException()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(LockTimeoutObject.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFastFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFastFinishFuture.java index 53d901aafb93a..435f2814af846 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFastFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFastFinishFuture.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -51,10 +52,15 @@ public class GridNearTxFastFinishFuture extends GridFutureAdapter implements NearTxFinishFuture { + /** */ + private final NearTxFinishFuture finishFut; + + /** + * @param finishFut Finish future. + */ + GridNearTxFinishAndAckFuture(NearTxFinishFuture finishFut) { + this.finishFut = finishFut; + } + + /** {@inheritDoc} */ + @Override public boolean commit() { + return finishFut.commit(); + } + + /** {@inheritDoc} */ + @Override public GridNearTxLocal tx() { + return finishFut.tx(); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + public void finish(boolean commit, boolean clearThreadMap, boolean onTimeout) { + finishFut.finish(commit, clearThreadMap, onTimeout); + + if (finishFut.commit()) { + finishFut.listen((IgniteInClosure)new IgniteInClosure() { + @Override public void apply(final NearTxFinishFuture fut) { + GridNearTxLocal tx = fut.tx(); + + IgniteInternalFuture ackFut = null; + + MvccQueryTracker tracker = tx.mvccQueryTracker(); + + MvccSnapshot mvccSnapshot = tx.mvccSnapshot(); + + if (tracker != null) + ackFut = tracker.onDone(tx, commit); + else if (mvccSnapshot != null) { + if (commit) + ackFut = tx.context().coordinators().ackTxCommit(mvccSnapshot); + else + tx.context().coordinators().ackTxRollback(mvccSnapshot); + } + + if (ackFut != null) { + ackFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture ackFut) { + Exception err = null; + + try { + fut.get(); + + ackFut.get(); + } + catch (Exception e) { + err = e; + } + catch (Error e) { + onDone(e); + + throw e; + } + + if (err != null) + onDone(err); + else + onDone(fut.tx()); + } + }); + } + else + finishWithFutureResult(fut); + } + }); + } + else { + finishFut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + finishWithFutureResult(fut); + } + }); + } + } + + /** {@inheritDoc} */ + @Override public void onNodeStop(IgniteCheckedException e) { + super.onDone(finishFut.tx(), e); + } + + /** + * @param fut Future. + */ + private void finishWithFutureResult(IgniteInternalFuture fut) { + try { + onDone(fut.get()); + } + catch (IgniteCheckedException | RuntimeException e) { + onDone(e); + } + catch (Error e) { + onDone(e); + + throw e; + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxFinishAndAckFuture.class, this); + } +} 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 297456350dff9..447d899ba54de 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 @@ -43,11 +43,15 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; +import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator; +import org.apache.ignite.internal.processors.cache.mvcc.MvccFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.C1; @@ -136,6 +140,13 @@ public GridNearTxFinishFuture(GridCacheSharedContext cctx, GridNearTxLocal return commit; } + /** + * @return Cache context. + */ + GridCacheSharedContext context() { + return cctx; + } + /** {@inheritDoc} */ @Override public IgniteUuid futureId() { return futId; @@ -390,15 +401,12 @@ private boolean isMini(IgniteInternalFuture fut) { fut.getClass() == CheckRemoteTxMiniFuture.class; } - /** - * Initializes future. - * - * @param commit Commit flag. - * @param clearThreadMap If {@code true} removes {@link GridNearTxLocal} from thread map. - * @param onTimeout If {@code true} called from timeout handler. - */ + /** {@inheritDoc} */ @SuppressWarnings("ForLoopReplaceableByForEach") public void finish(final boolean commit, final boolean clearThreadMap, final boolean onTimeout) { + if (!cctx.mvcc().addFuture(this, futureId())) + return; + if (tx.onNeedCheckBackup()) { assert tx.onePhaseCommit(); @@ -465,18 +473,38 @@ private void rollbackAsyncSafe(boolean onTimeout) { private void doFinish(boolean commit, boolean clearThreadMap) { try { if (tx.localFinish(commit, clearThreadMap) || (!commit && tx.state() == UNKNOWN)) { + GridLongList waitTxs = tx.mvccWaitTransactions(); + + if (waitTxs != null) { + MvccSnapshot snapshot = tx.mvccSnapshot(); + + MvccCoordinator crd = cctx.coordinators().currentCoordinator(); + + assert snapshot != null; + + if (snapshot.coordinatorVersion() == crd.coordinatorVersion()) { + IgniteInternalFuture fut = cctx.coordinators() + .waitTxsFuture(cctx.coordinators().currentCoordinatorId(), waitTxs); + + add(fut); + } + } + if ((tx.onePhaseCommit() && needFinishOnePhase(commit)) || (!tx.onePhaseCommit() && mappings != null)) { if (mappings.single()) { GridDistributedTxMapping mapping = mappings.singleMapping(); if (mapping != null) { - assert !hasFutures() : futures(); + assert !hasFutures() || waitTxs != null : futures(); finish(1, mapping, commit, !clearThreadMap); } } - else + else { + assert !hasFutures() || waitTxs != null : futures(); + finish(mappings.mappings(), commit, !clearThreadMap); + } } markInitialized(); @@ -501,6 +529,11 @@ private void doFinish(boolean commit, boolean clearThreadMap) { } } + /** {@inheritDoc} */ + @Override public void onNodeStop(IgniteCheckedException e) { + super.onDone(tx, e); + } + /** * */ @@ -732,8 +765,6 @@ private void readyNearMappingFromBackup(GridDistributedTxMapping mapping) { * @param {@code true} If need to add completed version on finish. */ private void finish(Iterable mappings, boolean commit, boolean useCompletedVer) { - assert !hasFutures() : futures(); - int miniId = 0; // Create mini futures. @@ -750,11 +781,11 @@ private void finish(Iterable mappings, boolean commit, private void finish(int miniId, GridDistributedTxMapping m, boolean commit, boolean useCompletedVer) { ClusterNode n = m.primary(); - assert !m.empty() : m + " " + tx.state(); + assert !m.empty() || m.queryUpdate() : m + " " + tx.state(); CacheWriteSynchronizationMode syncMode = tx.syncMode(); - if (m.explicitLock()) + if (m.explicitLock() || m.queryUpdate()) syncMode = FULL_SYNC; GridNearTxFinishRequest req = new GridNearTxFinishRequest( @@ -775,6 +806,7 @@ private void finish(int miniId, GridDistributedTxMapping m, boolean commit, bool tx.size(), tx.subjectId(), tx.taskNameHash(), + tx.mvccSnapshot(), tx.activeCachesDeploymentEnabled() ); @@ -869,6 +901,11 @@ else if (f.getClass() == CheckRemoteTxMiniFuture.class) { return "CheckRemoteTxMiniFuture[nodes=" + fut.nodes() + ", done=" + f.isDone() + "]"; } + else if (f instanceof MvccFuture) { + MvccFuture fut = (MvccFuture)f; + + return "WaitPreviousTxsFut[mvccCrd=" + fut.coordinatorNodeId() + ", done=" + f.isDone() + "]"; + } else return "[loc=true, done=" + f.isDone() + "]"; } @@ -909,7 +946,8 @@ private GridDhtTxFinishRequest checkCommittedRequest(int miniId, boolean waitRem 0, tx.activeCachesDeploymentEnabled(), !waitRemoteTxs && (tx.needReturnValue() && tx.implicit()), - waitRemoteTxs); + waitRemoteTxs, + null); finishReq.checkCommitted(true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java index 00c29e5f7b5c0..6b5aa90e3f5f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java @@ -24,6 +24,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishRequest; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.tostring.GridToStringBuilder; import org.apache.ignite.internal.util.typedef.internal.U; @@ -43,6 +44,9 @@ public class GridNearTxFinishRequest extends GridDistributedTxFinishRequest { /** Mini future ID. */ private int miniId; + /** */ + private MvccSnapshot mvccSnapshot; + /** * Empty constructor required for {@link Externalizable}. */ @@ -88,6 +92,7 @@ public GridNearTxFinishRequest( int txSize, @Nullable UUID subjId, int taskNameHash, + MvccSnapshot mvccSnapshot, boolean addDepInfo) { super( xidVer, @@ -111,6 +116,15 @@ public GridNearTxFinishRequest( explicitLock(explicitLock); storeEnabled(storeEnabled); + + this.mvccSnapshot = mvccSnapshot; + } + + /** + * @return Mvcc info. + */ + @Nullable public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; } /** @@ -178,6 +192,12 @@ public void miniId(int miniId) { writer.incrementState(); + case 22: + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + return false; + + writer.incrementState(); + } return true; @@ -202,6 +222,14 @@ public void miniId(int miniId) { reader.incrementState(); + case 22: + mvccSnapshot = reader.readMessage("mvccSnapshot"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridNearTxFinishRequest.class); @@ -214,7 +242,7 @@ public void miniId(int miniId) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 22; + return 23; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index a4acfa6c1f5a1..d03d0d3aaeffc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -47,10 +47,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; -import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; 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; @@ -61,6 +61,9 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtDetachedCacheEntry; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTrackerImpl; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -68,6 +71,7 @@ import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyRollbackOnlyImpl; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; @@ -134,6 +138,10 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou private static final AtomicReferenceFieldUpdater FINISH_FUT_UPD = AtomicReferenceFieldUpdater.newUpdater(GridNearTxLocal.class, NearTxFinishFuture.class, "finishFut"); + /** */ + private static final String TX_TYPE_MISMATCH_ERR_MSG = + "SQL queries and cache operations may not be used in the same transaction."; + /** DHT mappings. */ private IgniteTxMappings mappings; @@ -179,6 +187,15 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou /** Tx label. */ private @Nullable String lb; + /** */ + private MvccQueryTracker mvccTracker; + + /** Whether this transaction is for SQL operations or not.

    + * {@code null} means there haven't been any calls made on this transaction, and first operation will give this + * field actual value. + */ + private Boolean sql; + /** * Empty constructor required for {@link Externalizable}. */ @@ -196,6 +213,7 @@ public GridNearTxLocal() { * @param isolation Isolation. * @param timeout Timeout. * @param storeEnabled Store enabled flag. + * @param sql Whether this transaction was started via SQL API or not, or {@code null} if unknown. * @param txSize Transaction size. * @param subjId Subject ID. * @param taskNameHash Task name hash code. @@ -211,6 +229,7 @@ public GridNearTxLocal( TransactionIsolation isolation, long timeout, boolean storeEnabled, + Boolean sql, int txSize, @Nullable UUID subjId, int taskNameHash, @@ -237,11 +256,20 @@ public GridNearTxLocal( mappings = implicitSingle ? new IgniteTxMappingsSingleImpl() : new IgniteTxMappingsImpl(); + this.sql = sql; + initResult(); trackTimeout = timeout() > 0 && !implicit() && cctx.time().addTimeoutObject(this); } + /** + * @return Mvcc query version tracker. + */ + public MvccQueryTracker mvccQueryTracker() { + return mvccTracker; + } + /** {@inheritDoc} */ @Override public boolean near() { return true; @@ -546,7 +574,7 @@ private IgniteInternalFuture putAsync0( assert key != null; try { - beforePut(cacheCtx, retval); + beforePut(cacheCtx, retval, false); final GridCacheReturn ret = new GridCacheReturn(localResult(), false); @@ -689,7 +717,7 @@ private IgniteInternalFuture putAllAsync0( final boolean retval ) { try { - beforePut(cacheCtx, retval); + beforePut(cacheCtx, retval, false); } catch (IgniteCheckedException e) { return new GridFinishedFuture(e); @@ -855,6 +883,8 @@ private IgniteInternalFuture putAllAsync0( * @param ret Return value. * @param skipStore Skip store flag. * @param singleRmv {@code True} for single key remove operation ({@link Cache#remove(Object)}. + * @param recovery Recovery flag. + * @param dataCenterId Optional data center Id. * @return Future for entry values loading. */ private IgniteInternalFuture enlistWrite( @@ -978,6 +1008,7 @@ private IgniteInternalFuture enlistWrite( * @param skipStore Skip store flag. * @param singleRmv {@code True} for single key remove operation ({@link Cache#remove(Object)}. * @param keepBinary Keep binary flag. + * @param recovery Recovery flag. * @param dataCenterId Optional data center ID. * @return Future for enlisting writes. */ @@ -1245,6 +1276,7 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, resolveTaskName(), null, keepBinary, + null, // TODO IGNITE-7371 null) : null; if (res != null) { @@ -1263,7 +1295,8 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, entryProcessor, resolveTaskName(), null, - keepBinary); + keepBinary, + null); // TODO IGNITE-7371 } } catch (ClusterTopologyCheckedException e) { @@ -1524,6 +1557,9 @@ private IgniteInternalFuture removeAllAsync0( cacheCtx.checkSecurity(SecurityPermission.CACHE_REMOVE); + if (cacheCtx.mvccEnabled() && !isOperationAllowed(false)) + return txTypeMismatchFinishFuture(); + if (retval) needReturnValue(true); @@ -1731,6 +1767,109 @@ private IgniteInternalFuture removeAllAsync0( } } + /** + * @param cctx Cache context. + * @return Mvcc snapshot for read inside tx (initialized once for OPTIMISTIC SERIALIZABLE and REPEATABLE_READ txs). + */ + private MvccSnapshot mvccReadSnapshot(GridCacheContext cctx) { + if (!cctx.mvccEnabled() || mvccTracker == null) + return null; + + return mvccTracker.snapshot(); + } + + /** + * @param cacheCtx Cache context. + * @param cacheIds Involved cache ids. + * @param parts Partitions. + * @param schema Schema name. + * @param qry Query string. + * @param params Query parameters. + * @param flags Flags. + * @param pageSize Fetch page size. + * @param timeout Timeout. + * @return Operation future. + */ + public IgniteInternalFuture updateAsync(GridCacheContext cacheCtx, + int[] cacheIds, int[] parts, String schema, String qry, Object[] params, + int flags, int pageSize, long timeout) { + try { + beforePut(cacheCtx, false, true); + + return updateAsync(new GridNearTxQueryEnlistFuture( + cacheCtx, + this, + cacheIds, + parts, + schema, + qry, + params, + flags, + pageSize, + timeout)); + } + catch (IgniteCheckedException e) { + return new GridFinishedFuture(e); + } + catch (RuntimeException e) { + onException(); + + throw e; + } + } + + /** + * @param cacheCtx Cache context. + * @param it Entries iterator. + * @param pageSize Page size. + * @param timeout Timeout. + * @param sequential Sequential locking flag. + * @return Operation future. + */ + public IgniteInternalFuture updateAsync(GridCacheContext cacheCtx, + UpdateSourceIterator it, int pageSize, long timeout, boolean sequential) { + try { + beforePut(cacheCtx, false, true); + + return updateAsync(new GridNearTxQueryResultsEnlistFuture(cacheCtx, this, + timeout, it, pageSize, sequential)); + } + catch (IgniteCheckedException e) { + return new GridFinishedFuture(e); + } + catch (RuntimeException e) { + onException(); + + throw e; + } + } + + /** + * @param fut Enlist future. + * @return Operation future. + */ + public IgniteInternalFuture updateAsync(GridNearTxAbstractEnlistFuture fut) { + fut.init(); + + return nonInterruptable(new GridEmbeddedFuture<>(fut.chain(new CX1, Boolean>() { + @Override public Boolean applyx(IgniteInternalFuture fut0) throws IgniteCheckedException { + return fut0.get() != null; + } + }), new PLC1(null) { + @Override protected Long postLock(Long val) throws IgniteCheckedException { + Long res = fut.get(); + + assert mvccSnapshot != null; + assert res != null; + + if (res > 0) + mvccSnapshot.incrementOperationCounter(); + + return res; + } + })); + } + /** * @param cacheCtx Cache context. * @param keys Keys to get. @@ -1744,7 +1883,7 @@ private IgniteInternalFuture removeAllAsync0( public IgniteInternalFuture> getAllAsync( final GridCacheContext cacheCtx, @Nullable final AffinityTopologyVersion entryTopVer, - Collection keys, + final Collection keys, final boolean deserializeBinary, final boolean skipVals, final boolean keepCacheObjects, @@ -1754,8 +1893,36 @@ public IgniteInternalFuture> getAllAsync( if (F.isEmpty(keys)) return new GridFinishedFuture<>(Collections.emptyMap()); + if (cacheCtx.mvccEnabled() && !isOperationAllowed(false)) + return txTypeMismatchFinishFuture(); + init(); + if (cacheCtx.mvccEnabled() && (optimistic() && !readCommitted()) && mvccTracker == null) { + // TODO IGNITE-7388: support async tx rollback (e.g. on timeout). + boolean canRemap = cctx.lockedTopologyVersion(null) == null; + + mvccTracker = new MvccQueryTrackerImpl(cacheCtx, canRemap); + + return new GridEmbeddedFuture<>(mvccTracker.requestSnapshot(topologyVersion()), + new IgniteBiClosure>>() { + @Override public IgniteInternalFuture> apply(MvccSnapshot snapshot, Exception e) { + if (e != null) + return new GridFinishedFuture<>(e); + + return getAllAsync(cacheCtx, + entryTopVer, + keys, + deserializeBinary, + skipVals, + keepCacheObjects, + skipStore, + recovery, + needVer); + } + }); + } + int keysCnt = keys.size(); boolean single = keysCnt == 1; @@ -1878,6 +2045,7 @@ public IgniteInternalFuture> getAllAsync( resolveTaskName(), null, txEntry.keepBinary(), + null, // TODO IGNITE-7371 null); if (getRes != null) { @@ -1896,7 +2064,8 @@ public IgniteInternalFuture> getAllAsync( transformClo, resolveTaskName(), null, - txEntry.keepBinary()); + txEntry.keepBinary(), + null); // TODO IGNITE-7371 } // If value is in cache and passed the filter. @@ -2057,6 +2226,7 @@ public IgniteInternalFuture> getAllAsync( * @param skipVals Skip values flag. * @param keepCacheObjects Keep cache objects flag. * @param skipStore Skip store flag. + * @param recovery Recovery flag.. * @throws IgniteCheckedException If failed. * @return Enlisted keys. */ @@ -2175,7 +2345,8 @@ private Collection enlistRead( resolveTaskName(), null, txEntry.keepBinary(), - null); + null, + null); // TODO IGNITE-7371 if (getRes != null) { val = getRes.value(); @@ -2193,7 +2364,8 @@ private Collection enlistRead( transformClo, resolveTaskName(), null, - txEntry.keepBinary()); + txEntry.keepBinary(), + null); // TODO IGNITE-7371 } if (val != null) { @@ -2261,6 +2433,7 @@ private Collection enlistRead( resolveTaskName(), accessPlc, !deserializeBinary, + mvccReadSnapshot(cacheCtx), // TODO IGNITE-7371 null) : null; if (getRes != null) { @@ -2279,7 +2452,8 @@ private Collection enlistRead( null, resolveTaskName(), accessPlc, - !deserializeBinary); + !deserializeBinary, + mvccReadSnapshot(cacheCtx)); // TODO IGNITE-7371 } if (val != null) { @@ -2465,6 +2639,13 @@ private IgniteInternalFuture loadMissing( c); } + /** + * @return Finished future with error message about tx type mismatch. + */ + private static IgniteInternalFuture txTypeMismatchFinishFuture() { + return new GridFinishedFuture(new IgniteCheckedException(TX_TYPE_MISMATCH_ERR_MSG)); + } + /** * @param cacheCtx Cache context. * @param loadFut Missing keys load future. @@ -2551,7 +2732,7 @@ private void onException() { * @param expiryPlc Expiry policy. * @return Future with {@code True} value if loading took place. */ - public IgniteInternalFuture loadMissing( + private IgniteInternalFuture loadMissing( final GridCacheContext cacheCtx, AffinityTopologyVersion topVer, boolean readThrough, @@ -2610,7 +2791,8 @@ else if (cacheCtx.isColocated()) { skipVals, needVer, /*keepCacheObject*/true, - recovery + recovery, + mvccReadSnapshot(cacheCtx) ).chain(new C1, Void>() { @Override public Void apply(IgniteInternalFuture f) { try { @@ -2641,7 +2823,8 @@ else if (cacheCtx.isColocated()) { expiryPlc0, skipVals, needVer, - /*keepCacheObject*/true + /*keepCacheObject*/true, + mvccReadSnapshot(cacheCtx) ).chain(new C1>, Void>() { @Override public Void apply(IgniteInternalFuture> f) { try { @@ -2738,7 +2921,8 @@ private IgniteInternalFuture localCacheLoadMissing( resolveTaskName(), expiryPlc0, txEntry == null ? keepBinary : txEntry.keepBinary(), - null); + null, + null); // TODO IGNITE-7371 if (res == null) { if (misses == null) @@ -3158,7 +3342,7 @@ private void readyNearLock(IgniteTxEntry txEntry, if (state != COMMITTING && state != COMMITTED) throw isRollbackOnly() ? timedOut() ? timeoutException() : rollbackException() : new IgniteCheckedException("Invalid transaction state for commit [state=" + state() + - ", tx=" + this + ']'); + ", tx=" + this + ']'); else { if (log.isDebugEnabled()) log.debug("Invalid transaction state for commit (another thread is committing): " + this); @@ -3307,56 +3491,44 @@ public IgniteInternalFuture commitNearTxLocalAsync() { if (log.isDebugEnabled()) log.debug("Committing near local tx: " + this); - NearTxFinishFuture fut = finishFut; - - if (fut != null) - return chainFinishFuture(fut, true, true, false); - - if (fastFinish()) { - GridNearTxFastFinishFuture fut0; + final NearTxFinishFuture fut, fut0 = finishFut; boolean fastFinish; - if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFastFinishFuture(this, true))) - return chainFinishFuture(finishFut, true, true, false); - - fut0.finish(false); - - return fut0; - } - - final GridNearTxFinishFuture fut0; - - if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFinishFuture<>(cctx, this, true))) + if (fut0 != null || !FINISH_FUT_UPD.compareAndSet(this, null, fut = finishFuture(fastFinish = fastFinish(), true))) return chainFinishFuture(finishFut, true, true, false); - cctx.mvcc().addFuture(fut0, fut0.futureId()); - - final IgniteInternalFuture prepareFut = prepareNearTxLocal(); + if (!fastFinish) { + final IgniteInternalFuture prepareFut = prepareNearTxLocal(); - prepareFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture f) { - try { - // Make sure that here are no exceptions. - prepareFut.get(); + prepareFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture f) { + try { + // Make sure that here are no exceptions. + prepareFut.get(); - fut0.finish(true, true, false); - } - catch (Error | RuntimeException e) { - COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + fut.finish(true, true, false); + } + catch (Error | RuntimeException e) { + COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); - fut0.finish(false, true, false); + fut.finish(false, true, false); - throw e; - } - catch (IgniteCheckedException e) { - COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + throw e; + } + catch (IgniteCheckedException e) { + COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); - if (!(e instanceof NodeStoppingException)) - fut0.finish(false, true, true); + if (!(e instanceof NodeStoppingException)) + fut.finish(false, true, true); + else + fut.onNodeStop(e); + } } - } - }); + }); + } + else + fut.finish(true, false, false); - return fut0; + return fut; } /** {@inheritDoc} */ @@ -3400,61 +3572,70 @@ public IgniteInternalFuture rollbackNearTxLocalAsync(final boo if (onTimeout && prepFut instanceof GridNearTxPrepareFutureAdapter && !prepFut.isDone()) ((GridNearTxPrepareFutureAdapter) prepFut).onNearTxLocalTimeout(); - NearTxFinishFuture fut = finishFut; + final NearTxFinishFuture fut, fut0 = finishFut; boolean fastFinish; - if (fut != null) + if (fut0 != null) return chainFinishFuture(finishFut, false, clearThreadMap, onTimeout); - // Enable fast finish only from tx thread. - if (clearThreadMap && fastFinish()) { - GridNearTxFastFinishFuture fut0; - - if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFastFinishFuture(this, false))) - return chainFinishFuture(finishFut, false, true, onTimeout); - - fut0.finish(true); - - return fut0; - } - - final GridNearTxFinishFuture fut0; - - if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFinishFuture<>(cctx, this, false))) + if (!FINISH_FUT_UPD.compareAndSet(this, null, fut = finishFuture(fastFinish = clearThreadMap && fastFinish(), false))) return chainFinishFuture(finishFut, false, clearThreadMap, onTimeout); - cctx.mvcc().addFuture(fut0, fut0.futureId()); + rollbackFuture(fut); - if (prepFut == null || prepFut.isDone()) { - try { - // Check for errors in prepare future. - if (prepFut != null) - prepFut.get(); - } - catch (IgniteCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); + if (!fastFinish) { + if (prepFut == null || prepFut.isDone()) { + try { + // Check for errors in prepare future. + if (prepFut != null) + prepFut.get(); + } + catch (IgniteCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); + } + + fut.finish(false, clearThreadMap, onTimeout); } + else { + prepFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture f) { + try { + // Check for errors in prepare future. + f.get(); + } + catch (IgniteCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); + } - fut0.finish(false, clearThreadMap, onTimeout); - } - else { - prepFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture f) { - try { - // Check for errors in prepare future. - f.get(); - } - catch (IgniteCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); + fut.finish(false, clearThreadMap, onTimeout); } + }); + } + } + else + fut.finish(false, true, onTimeout); - fut0.finish(false, clearThreadMap, onTimeout); - } - }); + return fut; + } + + /** + * @return Transaction commit future. + * @param fast {@code True} in case of fast finish. + * @param commit {@code True} if commit. + */ + private NearTxFinishFuture finishFuture(boolean fast, boolean commit) { + NearTxFinishFuture fut = fast ? new GridNearTxFastFinishFuture(this, commit) : + new GridNearTxFinishFuture<>(cctx, this, commit); + + if (mvccQueryTracker() != null || mvccSnapshot != null || txState.mvccEnabled(cctx)) { + if (commit) + fut = new GridNearTxFinishAndAckFuture(fut); + else + fut.listen(new AckCoordinatorOnRollback(this)); } - return fut0; + return fut; } /** {@inheritDoc} */ @@ -3489,12 +3670,8 @@ private IgniteInternalFuture chainFinishFuture(final NearTxFin assert rollbackFut.isDone() : rollbackFut; } - else { - if (!cctx.mvcc().addFuture(rollbackFut, rollbackFut.futureId())) - return; - + else rollbackFut.finish(false, clearThreadMap, onTimeout); - } } else { finishFut.listen(new IgniteInClosure>() { @@ -3540,7 +3717,9 @@ private IgniteInternalFuture chainFinishFuture(final NearTxFin * @return {@code True} if 'fast finish' path can be used for transaction completion. */ private boolean fastFinish() { - return writeMap().isEmpty() && ((optimistic() && !serializable()) || readMap().isEmpty()); + return writeMap().isEmpty() + && ((optimistic() && !serializable()) || readMap().isEmpty()) + && (mappings.single() || F.view(mappings.mappings(), CU.FILTER_QUERY_MAPPING).isEmpty()); } /** @@ -3984,6 +4163,20 @@ public boolean hasRemoteLocks() { return hasRemoteLocks; } + /** + * @return {@code true} if this transaction does not have type flag set or it matches invoking operation, + * {@code false} otherwise. + */ + public boolean isOperationAllowed(boolean sqlOp) { + if (sql == null) { + sql = sqlOp; + + return true; + } + + return sql == sqlOp; + } + /** * @return Public API proxy. */ @@ -4185,13 +4378,19 @@ private boolean isAll(GridCacheContext cctx, /** * @param cacheCtx Cache context. * @param retval Return value flag. + * @param sql SQL operation flag. * @throws IgniteCheckedException If failed. */ - private void beforePut(GridCacheContext cacheCtx, boolean retval) throws IgniteCheckedException { + private void beforePut(GridCacheContext cacheCtx, boolean retval, boolean sql) throws IgniteCheckedException { + assert !sql || cacheCtx.mvccEnabled(); + checkUpdatesAllowed(cacheCtx); cacheCtx.checkSecurity(SecurityPermission.CACHE_PUT); + if (cacheCtx.mvccEnabled() && !isOperationAllowed(sql)) + throw new IgniteCheckedException(TX_TYPE_MISMATCH_ERR_MSG); + if (retval) needReturnValue(true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java index 5c0fcec8a0dde..a9b1848bb6d64 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java @@ -49,7 +49,7 @@ * Common code for tx prepare in optimistic and pessimistic modes. */ public abstract class GridNearTxPrepareFutureAdapter extends - GridCacheCompoundFuture implements GridCacheVersionedFuture { + GridCacheCompoundFuture implements GridCacheVersionedFuture { /** Logger reference. */ protected static final AtomicReference logRef = new AtomicReference<>(); @@ -58,9 +58,9 @@ public abstract class GridNearTxPrepareFutureAdapter extends AtomicReferenceFieldUpdater.newUpdater(GridNearTxPrepareFutureAdapter.class, Throwable.class, "err"); /** */ - private static final IgniteReducer REDUCER = - new IgniteReducer() { - @Override public boolean collect(GridNearTxPrepareResponse e) { + private static final IgniteReducer REDUCER = + new IgniteReducer() { + @Override public boolean collect(Object e) { return true; } @@ -165,7 +165,7 @@ public IgniteInternalTx tx() { * @param txMapping Transaction mapping. */ final void checkOnePhase(GridDhtTxMapping txMapping) { - if (tx.storeWriteThrough()) + if (tx.storeWriteThrough() || tx.txState().mvccEnabled(cctx)) // TODO IGNITE-3479 (onePhase + mvcc) return; Map> map = txMapping.transactionNodes(); @@ -255,7 +255,7 @@ else if (txEntry.cached().detached()) { txEntry.ttl(CU.toTtl(expiry.getExpiryForAccess())); } - if (!m.empty()) { + if (m.queryUpdate() || !m.empty()) { // This step is very important as near and DHT versions grow separately. cctx.versions().onReceived(nodeId, res.dhtVersion()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java index a6706093b11cd..55c809d6f189f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java @@ -58,6 +58,9 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest { /** */ private static final int ALLOW_WAIT_TOP_FUT_FLAG_MASK = 0x10; + /** */ + private static final int REQUEST_MVCC_CNTR_FLAG_MASK = 0x20; + /** Future ID. */ private IgniteUuid futId; @@ -149,6 +152,20 @@ public GridNearTxPrepareRequest( setFlag(allowWaitTopFut, ALLOW_WAIT_TOP_FUT_FLAG_MASK); } + /** + * @return {@code True} if need request MVCC counter on primary node on prepare step. + */ + public boolean requestMvccCounter() { + return isFlag(REQUEST_MVCC_CNTR_FLAG_MASK); + } + + /** + * @param val {@code True} if need request MVCC counter on primary node on prepare step. + */ + public void requestMvccCounter(boolean val) { + setFlag(val, REQUEST_MVCC_CNTR_FLAG_MASK); + } + /** * @return {@code True} if it is safe for first client request to wait for topology future * completion. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java index 8162168136d9d..92db4633b732a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -97,6 +98,9 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse /** Not {@code null} if client node should remap transaction. */ private AffinityTopologyVersion clientRemapVer; + /** */ + private MvccSnapshot mvccSnapshot; + /** * Empty constructor required by {@link Externalizable}. */ @@ -145,6 +149,20 @@ public GridNearTxPrepareResponse( flags |= NEAR_PREPARE_ONE_PHASE_COMMIT_FLAG_MASK; } + /** + * @param mvccSnapshot Mvcc info. + */ + public void mvccSnapshot(MvccSnapshot mvccSnapshot) { + this.mvccSnapshot = mvccSnapshot; + } + + /** + * @return Mvcc info. + */ + @Nullable public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + /** * @return One-phase commit state on primary node. */ @@ -389,30 +407,36 @@ public boolean hasOwnedValue(IgniteTxKey key) { writer.incrementState(); case 15: - if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 16: - if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 17: - if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 18: - if (!writer.writeMessage("retVal", retVal)) + if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 19: + if (!writer.writeMessage("retVal", retVal)) + return false; + + writer.incrementState(); + + case 20: if (!writer.writeMessage("writeVer", writeVer)) return false; @@ -475,7 +499,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 15: - ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -483,7 +507,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 16: - ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG); + ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -491,7 +515,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 17: - pending = reader.readCollection("pending", MessageCollectionItemType.MSG); + ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -499,7 +523,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 18: - retVal = reader.readMessage("retVal"); + pending = reader.readCollection("pending", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -507,6 +531,14 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 19: + retVal = reader.readMessage("retVal"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 20: writeVer = reader.readMessage("writeVer"); if (!reader.isLastRead()) @@ -526,7 +558,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 20; + return 21; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java new file mode 100644 index 0000000000000..21967d714946c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java @@ -0,0 +1,382 @@ +/* + * 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.near; + +import java.util.Collection; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityAssignment; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryEnlistFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.cache.distributed.dht.NearTxQueryEnlistResultHandler.createResponse; + +/** + * Cache lock future. + */ +@SuppressWarnings("ForLoopReplaceableByForEach") +public class GridNearTxQueryEnlistFuture extends GridNearTxAbstractEnlistFuture { + /** */ + private static final long serialVersionUID = -2155104765461405820L; + /** Involved cache ids. */ + private final int[] cacheIds; + + /** Partitions. */ + private final int[] parts; + + /** Schema name. */ + private final String schema; + + /** Query string. */ + private final String qry; + + /** Query parameters. */ + private final Object[] params; + + /** Flags. */ + private final int flags; + + /** Fetch page size. */ + private final int pageSize; + + /** + * @param cctx Cache context. + * @param tx Transaction. + * @param cacheIds Involved cache ids. + * @param parts Partitions. + * @param schema Schema name. + * @param qry Query string. + * @param params Query parameters. + * @param flags Flags. + * @param pageSize Fetch page size. + * @param timeout Timeout. + */ + protected GridNearTxQueryEnlistFuture( + GridCacheContext cctx, GridNearTxLocal tx, int[] cacheIds, int[] parts, String schema, String qry, + Object[] params, int flags, int pageSize, long timeout) { + super(cctx, tx, timeout); + + this.cacheIds = cacheIds; + this.parts = parts; + this.schema = schema; + this.qry = qry; + this.params = params; + this.flags = flags; + this.pageSize = pageSize; + } + + /** + * @param topLocked Topology locked flag. + */ + protected void map(final boolean topLocked) { + MiniFuture mini = null; + + try { + final AffinityAssignment assignment = cctx.affinity().assignment(topVer); + + Collection primary; + + if (parts != null) { + primary = U.newHashSet(parts.length); + + for (int i = 0; i < parts.length; i++) { + ClusterNode pNode = assignment.get(parts[i]).get(0); + + primary.add(pNode); + + updateMappings(pNode); + } + } + else { + primary = assignment.primaryPartitionNodes(); + + for (ClusterNode pNode : primary) { + updateMappings(pNode); + } + } + + boolean locallyMapped = primary.contains(cctx.localNode()); + + if (locallyMapped) + add(new MiniFuture(cctx.localNode())); + + int idx = locallyMapped ? 1 : 0; + boolean first = true; + boolean clientFirst = false; + + for (ClusterNode node : F.view(primary, F.remoteNodes(cctx.localNodeId()))) { + add(mini = new MiniFuture(node)); + + if (first) { + clientFirst = cctx.localNode().isClient() && !topLocked && !tx.hasRemoteLocks(); + + first = false; + } + + GridNearTxQueryEnlistRequest req = new GridNearTxQueryEnlistRequest( + cctx.cacheId(), + threadId, + futId, + ++idx, + tx.subjectId(), + topVer, + lockVer, + mvccSnapshot, + cacheIds, + parts, + schema, + qry, + params, + flags, + pageSize, + remainingTime(), + tx.remainingTime(), + tx.taskNameHash(), + clientFirst + ); + + sendRequest(req, node.id(), mini); + } + + if (locallyMapped) { + final MiniFuture localMini = mini = miniFuture(-1); + + assert localMini != null; + + GridDhtTxQueryEnlistFuture fut = new GridDhtTxQueryEnlistFuture( + cctx.localNode().id(), + lockVer, + mvccSnapshot, + threadId, + futId, + -1, + tx, + cacheIds, + parts, + schema, + qry, + params, + flags, + pageSize, + remainingTime(), + cctx); + + updateLocalFuture(fut); + + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + assert fut.error() != null || fut.result() != null : fut; + + try { + clearLocalFuture((GridDhtTxQueryEnlistFuture)fut); + + GridNearTxQueryEnlistResponse res = fut.error() == null ? createResponse(fut) : null; + + localMini.onResult(res, fut.error()); + } + catch (IgniteCheckedException e) { + localMini.onResult(null, e); + } + finally { + CU.unwindEvicts(cctx); + } + } + }); + + fut.init(); + } + } + catch (Throwable e) { + if (mini != null) + mini.onResult(null, e); + else + onDone(e); + + if (e instanceof Error) + throw (Error)e; + } + + markInitialized(); + } + + /** + * + * @param req Request. + * @param nodeId Remote node ID. + * @param fut Result future. + * @throws IgniteCheckedException if failed to send. + */ + private void sendRequest(GridCacheMessage req, UUID nodeId, MiniFuture fut) throws IgniteCheckedException { + IgniteInternalFuture txSync = cctx.tm().awaitFinishAckAsync(nodeId, tx.threadId()); + + if (txSync == null || txSync.isDone()) + cctx.io().send(nodeId, req, cctx.ioPolicy()); + else + txSync.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture f) { + try { + cctx.io().send(nodeId, req, cctx.ioPolicy()); + } + catch (IgniteCheckedException e) { + fut.onResult(null, e); + } + } + }); + } + + /** + * @param nodeId Left node ID + * @return {@code True} if node was in the list. + */ + @Override public synchronized boolean onNodeLeft(UUID nodeId) { + for (IgniteInternalFuture fut : futures()) { + MiniFuture f = (MiniFuture)fut; + + if (f.node.id().equals(nodeId)) { + if (log.isDebugEnabled()) + log.debug("Found mini-future for left node [nodeId=" + nodeId + ", mini=" + f + ", fut=" + + this + ']'); + + return f.onResult(null, newTopologyException(nodeId)); + } + } + + if (log.isDebugEnabled()) + log.debug("Future does not have mapping for left node (ignoring) [nodeId=" + nodeId + + ", fut=" + this + ']'); + + return false; + } + + /** + * Finds pending mini future by the given mini ID. + * + * @param miniId Mini ID to find. + * @return Mini future. + */ + private MiniFuture miniFuture(int miniId) { + synchronized (this) { + int idx = Math.abs(miniId) - 1; + + assert idx >= 0 && idx < futuresCountNoLock(); + + IgniteInternalFuture fut = future(idx); + + if (!fut.isDone()) + return (MiniFuture)fut; + } + + return null; + } + + /** + * Creates new topology exception for cases when primary node leaves grid during mapping. + * + * @param nodeId Node ID. + * @return Topology exception with user-friendly message. + */ + private ClusterTopologyCheckedException newTopologyException(UUID nodeId) { + ClusterTopologyCheckedException topEx = new ClusterTopologyCheckedException("Failed to enlist keys " + + "(primary node left grid, retry transaction if possible) [node=" + nodeId + ']'); + + topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer)); + + return topEx; + } + + /** + * @param nodeId Sender node id. + * @param res Response. + */ + public void onResult(UUID nodeId, GridNearTxQueryEnlistResponse res) { + MiniFuture mini = miniFuture(res.miniId()); + + if (mini != null) + mini.onResult(res, null); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxQueryEnlistFuture.class, this, super.toString()); + } + + /** */ + private class MiniFuture extends GridFutureAdapter { + /** */ + private boolean completed; + + /** Node ID. */ + @GridToStringExclude + private final ClusterNode node; + + /** + * @param node Cluster node. + */ + private MiniFuture(ClusterNode node) { + this.node = node; + } + + /** + * @param res Response. + * @param err Exception. + * @return {@code True} if future was completed by this call. + */ + public boolean onResult(GridNearTxQueryEnlistResponse res, Throwable err) { + assert res != null || err != null : this; + + if (err == null && res.error() != null) + err = res.error(); + + synchronized (this) { + if (completed) + return false; + + completed = true; + } + + if (X.hasCause(err, ClusterTopologyCheckedException.class) + || (res != null && res.removeMapping())) { + GridDistributedTxMapping m = tx.mappings().get(node.id()); + + assert m != null && m.empty(); + + tx.removeMapping(node.id()); + + if (node.isLocal()) + tx.colocatedLocallyMapped(false); + } + else if (res != null && res.result() > 0 && !node.isLocal()) + tx.hasRemoteLocks(true); + + return err != null ? onDone(err) : onDone(res.result(), res.error()); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java new file mode 100644 index 0000000000000..472937be41569 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistRequest.java @@ -0,0 +1,616 @@ +/* + * 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.near; + +import java.nio.ByteBuffer; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +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.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * + */ +public class GridNearTxQueryEnlistRequest extends GridCacheIdMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long threadId; + + /** */ + private IgniteUuid futId; + + /** */ + private boolean clientFirst; + + /** */ + private int miniId; + + /** */ + private UUID subjId; + + /** */ + private AffinityTopologyVersion topVer; + + /** */ + private GridCacheVersion lockVer; + + /** */ + private MvccSnapshot mvccSnapshot; + + /** */ + private int[] cacheIds; + + /** */ + private int[] parts; + + /** */ + private String schema; + + /** */ + private String qry; + + /** */ + @GridDirectTransient + private Object[] params; + + /** */ + private byte[] paramsBytes; + + /** */ + private int flags; + + /** */ + private long timeout; + + /** */ + private long txTimeout; + + /** */ + private int taskNameHash; + + /** */ + private int pageSize; + + /** */ + public GridNearTxQueryEnlistRequest() { + // No-op. + } + + /** + * @param cacheId Cache id. + * @param threadId Thread id. + * @param futId Future id. + * @param miniId Mini fture id. + * @param subjId Subject id. + * @param topVer Topology version. + * @param lockVer Lock version. + * @param mvccSnapshot MVCC snspshot. + * @param cacheIds Involved cache ids. + * @param parts Partitions. + * @param schema Schema name. + * @param qry Query string. + * @param params Query parameters. + * @param flags Flags. + * @param pageSize Fetch page size. + * @param timeout Timeout milliseconds. + * @param txTimeout Tx timeout milliseconds. + * @param taskNameHash Task name hash. + * @param clientFirst {@code True} if this is the first client request. + */ + public GridNearTxQueryEnlistRequest( + int cacheId, + long threadId, + IgniteUuid futId, + int miniId, + UUID subjId, + AffinityTopologyVersion topVer, + GridCacheVersion lockVer, + MvccSnapshot mvccSnapshot, + int[] cacheIds, + int[] parts, + String schema, + String qry, + Object[] params, + int flags, + int pageSize, + long timeout, + long txTimeout, + int taskNameHash, + boolean clientFirst) { + this.cacheIds = cacheIds; + this.parts = parts; + this.schema = schema; + this.qry = qry; + this.params = params; + this.flags = flags; + this.pageSize = pageSize; + this.txTimeout = txTimeout; + this.cacheId = cacheId; + this.threadId = threadId; + this.futId = futId; + this.miniId = miniId; + this.subjId = subjId; + this.topVer = topVer; + this.lockVer = lockVer; + this.mvccSnapshot = mvccSnapshot; + this.timeout = timeout; + this.taskNameHash = taskNameHash; + this.clientFirst = clientFirst; + } + + /** + * @return Thread id. + */ + public long threadId() { + return threadId; + } + + /** + * @return Future id. + */ + public IgniteUuid futureId() { + return futId; + } + + /** + * @return Mini future ID. + */ + public int miniId() { + return miniId; + } + + /** + * @return Subject id. + */ + public UUID subjectId() { + return subjId; + } + + /** + * @return Topology version. + */ + @Override public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** + * @return Lock version. + */ + public GridCacheVersion version() { + return lockVer; + } + + /** + * @return MVCC snapshot. + */ + public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + + /** + * @return Involved cache ids. + */ + public int[] cacheIds() { + return cacheIds; + } + + /** + * @return Partitions. + */ + public int[] partitions() { + return parts; + } + + /** + * @return Schema name. + */ + public String schemaName() { + return schema; + } + + /** + * @return Query string. + */ + public String query() { + return qry; + } + + /** + * @return Query parameters. + */ + public Object[] parameters() { + return params; + } + + /** + * @return Flags. + */ + public int flags() { + return flags; + } + + /** + * @return Fetch page size. + */ + public int pageSize() { + return pageSize; + } + + /** + * @return Timeout milliseconds. + */ + public long timeout() { + return timeout; + } + + /** + * @return Tx timeout milliseconds. + */ + public long txTimeout() { + return txTimeout; + } + + /** + * @return Task name hash. + */ + public int taskNameHash() { + return taskNameHash; + } + + /** + * @return {@code True} if this is the first client request. + */ + public boolean firstClientRequest() { + return clientFirst; + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return false; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 21; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + if (params != null && paramsBytes == null) + paramsBytes = U.marshal(ctx, params); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + if (paramsBytes != null && params == null) + params = U.unmarshal(ctx, paramsBytes, ldr); + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeIntArray("cacheIds", cacheIds)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeBoolean("clientFirst", clientFirst)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeInt("flags", flags)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeIgniteUuid("futId", futId)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeMessage("lockVer", lockVer)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeInt("miniId", miniId)) + return false; + + writer.incrementState(); + + case 9: + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeInt("pageSize", pageSize)) + return false; + + writer.incrementState(); + + case 11: + if (!writer.writeByteArray("paramsBytes", paramsBytes)) + return false; + + writer.incrementState(); + + case 12: + if (!writer.writeIntArray("parts", parts)) + return false; + + writer.incrementState(); + + case 13: + if (!writer.writeString("qry", qry)) + return false; + + writer.incrementState(); + + case 14: + if (!writer.writeString("schema", schema)) + return false; + + writer.incrementState(); + + case 15: + if (!writer.writeUuid("subjId", subjId)) + return false; + + writer.incrementState(); + + case 16: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 17: + if (!writer.writeLong("threadId", threadId)) + return false; + + writer.incrementState(); + + case 18: + if (!writer.writeLong("timeout", timeout)) + return false; + + writer.incrementState(); + + case 19: + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + case 20: + if (!writer.writeLong("txTimeout", txTimeout)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + cacheIds = reader.readIntArray("cacheIds"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + clientFirst = reader.readBoolean("clientFirst"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + flags = reader.readInt("flags"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + futId = reader.readIgniteUuid("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + lockVer = reader.readMessage("lockVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: + miniId = reader.readInt("miniId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 9: + mvccSnapshot = reader.readMessage("mvccSnapshot"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: + pageSize = reader.readInt("pageSize"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: + paramsBytes = reader.readByteArray("paramsBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 12: + parts = reader.readIntArray("parts"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 13: + qry = reader.readString("qry"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 14: + schema = reader.readString("schema"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 15: + subjId = reader.readUuid("subjId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 16: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 17: + threadId = reader.readLong("threadId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 18: + timeout = reader.readLong("timeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 19: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 20: + txTimeout = reader.readLong("txTimeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridNearTxQueryEnlistRequest.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 151; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxQueryEnlistRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java new file mode 100644 index 0000000000000..39e33c46b8243 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistResponse.java @@ -0,0 +1,298 @@ +/* + * 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.near; + +import java.nio.ByteBuffer; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.ExceptionAware; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +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.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class GridNearTxQueryEnlistResponse extends GridCacheIdMessage implements ExceptionAware { + /** */ + private static final long serialVersionUID = 0L; + + /** Future ID. */ + private IgniteUuid futId; + + /** Error. */ + @GridDirectTransient + private Throwable err; + + /** Serialized error. */ + private byte[] errBytes; + + /** Mini future id. */ + private int miniId; + + /** Result. */ + private long res; + + /** Remove mapping flag. */ + private boolean removeMapping; + + /** */ + private GridCacheVersion lockVer; + + /** + * Default constructor. + */ + public GridNearTxQueryEnlistResponse() { + // No-op. + } + + /** + * @param cacheId Cache id. + * @param futId Future id. + * @param miniId Mini future id. + * @param lockVer Lock version. + * @param err Error. + */ + public GridNearTxQueryEnlistResponse(int cacheId, IgniteUuid futId, int miniId, GridCacheVersion lockVer, Throwable err) { + this.cacheId = cacheId; + this.futId = futId; + this.miniId = miniId; + this.lockVer = lockVer; + this.err = err; + } + + /** + * @param cacheId Cache id. + * @param futId Future id. + * @param miniId Mini future id. + * @param lockVer Lock version. + * @param res Result. + * @param removeMapping Remove mapping flag. + */ + public GridNearTxQueryEnlistResponse(int cacheId, IgniteUuid futId, int miniId, GridCacheVersion lockVer, long res, boolean removeMapping) { + this.cacheId = cacheId; + this.futId = futId; + this.miniId = miniId; + this.lockVer = lockVer; + this.res = res; + this.removeMapping = removeMapping; + } + + /** + * @return Loc version. + */ + public GridCacheVersion version() { + return lockVer; + } + + /** + * @return Future id. + */ + public IgniteUuid futureId() { + return futId; + } + + /** + * @return Mini future id. + */ + public int miniId() { + return miniId; + } + + /** + * @return Result. + */ + public long result() { + return res; + } + + /** + * @return Remove mapping flag. + */ + public boolean removeMapping() { + return removeMapping; + } + + /** {@inheritDoc} */ + @Nullable @Override public Throwable error() { + return err; + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return false; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 9; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeByteArray("errBytes", errBytes)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeIgniteUuid("futId", futId)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeMessage("lockVer", lockVer)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeInt("miniId", miniId)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeBoolean("removeMapping", removeMapping)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeLong("res", res)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + errBytes = reader.readByteArray("errBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + futId = reader.readIgniteUuid("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + lockVer = reader.readMessage("lockVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + miniId = reader.readInt("miniId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + removeMapping = reader.readBoolean("removeMapping"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: + res = reader.readLong("res"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridNearTxQueryEnlistResponse.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 152; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + if (err != null && errBytes == null) + errBytes = U.marshal(ctx.marshaller(), err); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + if (errBytes != null) + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxQueryEnlistResponse.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java new file mode 100644 index 0000000000000..910deb983ef5b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java @@ -0,0 +1,670 @@ +/* + * 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.near; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryResultsEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemote; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotWithoutTxs; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; +import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.distributed.dht.NearTxQueryEnlistResultHandler.createResponse; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * A future tracking requests for remote nodes transaction enlisting and locking + * of entries produced with complex DML queries requiring reduce step. + */ +public class GridNearTxQueryResultsEnlistFuture extends GridNearTxAbstractEnlistFuture { + /** */ + private static final long serialVersionUID = 4339957209840477447L; + + /** */ + public static final int DFLT_BATCH_SIZE = 1024; + + /** Res field updater. */ + private static final AtomicLongFieldUpdater RES_UPD = + AtomicLongFieldUpdater.newUpdater(GridNearTxQueryResultsEnlistFuture.class, "res"); + + /** SkipCntr field updater. */ + private static final AtomicIntegerFieldUpdater SKIP_UPD = + AtomicIntegerFieldUpdater.newUpdater(GridNearTxQueryResultsEnlistFuture.class, "skipCntr"); + + /** Marker object. */ + private static final Object FINISHED = new Object(); + + /** */ + private final UpdateSourceIterator it; + + /** */ + private int batchSize; + + /** */ + private AtomicInteger batchCntr = new AtomicInteger(); + + /** */ + @SuppressWarnings("unused") + @GridToStringExclude + private volatile int skipCntr; + + /** */ + @SuppressWarnings("unused") + @GridToStringExclude + private volatile long res; + + /** */ + private final Map batches = new ConcurrentHashMap<>(); + + /** Row extracted from iterator but not yet used. */ + private Object peek; + + /** Topology locked flag. */ + private boolean topLocked; + + /** */ + private final boolean sequential; + + /** + * @param cctx Cache context. + * @param tx Transaction. + * @param timeout Timeout. + * @param it Rows iterator. + * @param batchSize Batch size. + * @param sequential Sequential locking flag. + */ + public GridNearTxQueryResultsEnlistFuture(GridCacheContext cctx, + GridNearTxLocal tx, + long timeout, + UpdateSourceIterator it, + int batchSize, + boolean sequential) { + super(cctx, tx, timeout); + + this.it = it; + this.batchSize = batchSize > 0 ? batchSize : DFLT_BATCH_SIZE; + this.sequential = sequential; + } + + /** {@inheritDoc} */ + @Override protected void map(boolean topLocked) { + this.topLocked = topLocked; + + sendNextBatches(null); + } + + /** + * Continue iterating the data rows and form new batches. + * + * @param nodeId Node that is ready for a new batch. + */ + private void sendNextBatches(@Nullable UUID nodeId) { + try { + Collection next = continueLoop(nodeId); + + if (next == null) + return; + + boolean first = (nodeId != null); + + for (Batch batch : next) { + ClusterNode node = batch.node(); + + sendBatch(node, batch, first); + + if (!node.isLocal()) + first = false; + } + } + catch (Throwable e) { + onDone(e); + + if (e instanceof Error) + throw (Error)e; + } + } + + /** + * Iterate data rows and form batches. + * + * @param nodeId Id of node acknowledged the last batch. + * @return Collection of newly completed batches. + * @throws IgniteCheckedException If failed. + */ + private Collection continueLoop(@Nullable UUID nodeId) throws IgniteCheckedException { + if (nodeId != null) + batches.remove(nodeId); + + // Accumulate number of batches released since we got here. + // Let only one thread do the looping. + if (isDone() || SKIP_UPD.getAndIncrement(this) != 0) + return null; + + ArrayList res = null; Batch batch = null; + + boolean flush = false; + + EnlistOperation op = it.operation(); + + while (true) { + while (hasNext0()) { + checkCompleted(); + + Object cur = next0(); + + KeyCacheObject key = cctx.toCacheKeyObject(op.isDeleteOrLock() ? cur : ((IgniteBiTuple)cur).getKey()); + + List nodes = cctx.affinity().nodesByKey(key, topVer); + + ClusterNode node; + + if (F.isEmpty(nodes) || ((node = nodes.get(0)) == null)) + throw new ClusterTopologyCheckedException("Failed to get primary node " + + "[topVer=" + topVer + ", key=" + key + ']'); + + if (!sequential) + batch = batches.get(node.id()); + else if (batch != null && !batch.node().equals(node)) + res = markReady(res, batch); + + if (batch == null) + batches.put(node.id(), batch = new Batch(node)); + + if (batch.ready()) { + // Can't advance further at the moment. + batch = null; + + peek = cur; + + it.beforeDetach(); + + flush = true; + + break; + } + + batch.add(op.isDeleteOrLock() ? key : cur, + op != EnlistOperation.LOCK && cctx.affinityNode() && (cctx.isReplicated() || nodes.indexOf(cctx.localNode()) > 0)); + + if (batch.size() == batchSize) + res = markReady(res, batch); + } + + if (SKIP_UPD.decrementAndGet(this) == 0) + break; + + skipCntr = 1; + } + + if (flush) + return res; + + // No data left - flush incomplete batches. + for (Batch batch0 : batches.values()) { + if (!batch0.ready()) { + if (res == null) + res = new ArrayList<>(); + + batch0.ready(true); + + res.add(batch0); + } + } + + if (batches.isEmpty()) + onDone(this.res); + + return res; + } + + /** */ + private Object next0() { + if (!hasNext0()) + throw new NoSuchElementException(); + + Object cur; + + if ((cur = peek) != null) + peek = null; + else + cur = it.next(); + + return cur; + } + + /** */ + private boolean hasNext0() { + if (peek == null && !it.hasNext()) + peek = FINISHED; + + return peek != FINISHED; + } + + /** */ + private ArrayList markReady(ArrayList batches, Batch batch) { + if (!batch.ready()) { + batch.ready(true); + + if (batches == null) + batches = new ArrayList<>(); + + batches.add(batch); + } + + return batches; + } + + /** + * + * @param primaryId Primary node id. + * @param rows Rows. + * @param dhtVer Dht version assigned at primary node. + * @param dhtFutId Dht future id assigned at primary node. + * @param updCntrs Update counters. + */ + private void processBatchLocalBackupKeys(UUID primaryId, List rows, GridCacheVersion dhtVer, + IgniteUuid dhtFutId, GridLongList updCntrs) { + assert dhtVer != null; + assert dhtFutId != null; + + EnlistOperation op = it.operation(); + + assert op != EnlistOperation.LOCK; + + boolean keysOnly = op.isDeleteOrLock(); + + final ArrayList keys = new ArrayList<>(rows.size()); + final ArrayList vals = keysOnly ? null : new ArrayList<>(rows.size()); + + for (Object row : rows) { + if (keysOnly) + keys.add(cctx.toCacheKeyObject(row)); + else { + keys.add(cctx.toCacheKeyObject(((IgniteBiTuple)row).getKey())); + vals.add(cctx.toCacheObject(((IgniteBiTuple)row).getValue())); + } + } + + try { + GridDhtTxRemote dhtTx = cctx.tm().tx(dhtVer); + + if (dhtTx == null) { + dhtTx = new GridDhtTxRemote(cctx.shared(), + cctx.localNodeId(), + dhtFutId, + primaryId, + lockVer, + topVer, + dhtVer, + null, + cctx.systemTx(), + cctx.ioPolicy(), + PESSIMISTIC, + REPEATABLE_READ, + false, + tx.remainingTime(), + -1, + this.tx.subjectId(), + this.tx.taskNameHash(), + false); + + dhtTx.mvccSnapshot(new MvccSnapshotWithoutTxs(mvccSnapshot.coordinatorVersion(), + mvccSnapshot.counter(), MVCC_OP_COUNTER_NA, mvccSnapshot.cleanupVersion())); + + dhtTx = cctx.tm().onCreated(null, dhtTx); + + if (dhtTx == null || !cctx.tm().onStarted(dhtTx)) { + throw new IgniteTxRollbackCheckedException("Failed to update backup " + + "(transaction has been completed): " + dhtVer); + } + } + + dhtTx.mvccEnlistBatch(cctx, it.operation(), keys, vals, mvccSnapshot.withoutActiveTransactions(), updCntrs); + } + catch (IgniteCheckedException e) { + onDone(e); + + return; + } + + sendNextBatches(primaryId); + } + + /** + * + * @param node Node. + * @param batch Batch. + * @param first First mapping flag. + */ + private void sendBatch(ClusterNode node, Batch batch, boolean first) throws IgniteCheckedException { + updateMappings(node); + + boolean clientFirst = first && cctx.localNode().isClient() && !topLocked && !tx.hasRemoteLocks(); + + int batchId = batchCntr.incrementAndGet(); + + if (node.isLocal()) + enlistLocal(batchId, node.id(), batch); + else + sendBatch(batchId, node.id(), batch, clientFirst); + } + + /** + * Send batch request to remote data node. + * + * @param batchId Id of a batch mini-future. + * @param nodeId Node id. + * @param batchFut Mini-future for the batch. + * @param clientFirst {@code true} if originating node is client and it is a first request to any data node. + */ + private void sendBatch(int batchId, UUID nodeId, Batch batchFut, boolean clientFirst) throws IgniteCheckedException { + assert batchFut != null; + + GridNearTxQueryResultsEnlistRequest req = new GridNearTxQueryResultsEnlistRequest(cctx.cacheId(), + threadId, + futId, + batchId, + tx.subjectId(), + topVer, + lockVer, + mvccSnapshot, + clientFirst, + remainingTime(), + tx.remainingTime(), + tx.taskNameHash(), + batchFut.rows(), + it.operation()); + + sendRequest(req, nodeId); + } + + /** + * + * @param req Request. + * @param nodeId Remote node ID + * @throws IgniteCheckedException if failed to send. + */ + private void sendRequest(GridCacheMessage req, UUID nodeId) throws IgniteCheckedException { + IgniteInternalFuture txSync = cctx.tm().awaitFinishAckAsync(nodeId, tx.threadId()); + + if (txSync == null || txSync.isDone()) + cctx.io().send(nodeId, req, cctx.ioPolicy()); + else + txSync.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture future) { + try { + cctx.io().send(nodeId, req, cctx.ioPolicy()); + } + catch (IgniteCheckedException e) { + GridNearTxQueryResultsEnlistFuture.this.onDone(e); + } + } + }); + } + + /** + * Enlist batch of entries to the transaction on local node. + * + * @param batchId Id of a batch mini-future. + * @param nodeId Node id. + * @param batch Batch. + */ + private void enlistLocal(int batchId, UUID nodeId, Batch batch) throws IgniteCheckedException { + Collection rows = batch.rows(); + + GridDhtTxQueryResultsEnlistFuture fut = new GridDhtTxQueryResultsEnlistFuture(nodeId, + lockVer, + mvccSnapshot, + threadId, + futId, + batchId, + tx, + remainingTime(), + cctx, + rows, + it.operation()); + + updateLocalFuture(fut); + + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + assert fut.error() != null || fut.result() != null : fut; + + try { + clearLocalFuture((GridDhtTxAbstractEnlistFuture)fut); + + GridNearTxQueryResultsEnlistResponse res = fut.error() == null ? createResponse(fut) : null; + + if (checkResponse(nodeId, res, fut.error())) + sendNextBatches(nodeId); + } + catch (IgniteCheckedException e) { + checkResponse(nodeId, null, e); + } + finally { + CU.unwindEvicts(cctx); + } + } + }); + + fut.init(); + } + + /** + * @param nodeId Sender node id. + * @param res Response. + */ + public void onResult(UUID nodeId, GridNearTxQueryResultsEnlistResponse res) { + if (checkResponse(nodeId, res, res.error())) { + + Batch batch = batches.get(nodeId); + + if (batch != null && !F.isEmpty(batch.localBackupRows()) && res.dhtFutureId() != null) + processBatchLocalBackupKeys(nodeId, batch.localBackupRows(), res.dhtVersion(), res.dhtFutureId(), + res.updateCounters()); + else + sendNextBatches(nodeId); + } + } + + /** {@inheritDoc} */ + @Override public boolean onNodeLeft(UUID nodeId) { + if (batches.keySet().contains(nodeId)) { + if (log.isDebugEnabled()) + log.debug("Found unacknowledged batch for left node [nodeId=" + nodeId + ", fut=" + + this + ']'); + + ClusterTopologyCheckedException topEx = new ClusterTopologyCheckedException("Failed to enlist keys " + + "(primary node left grid, retry transaction if possible) [node=" + nodeId + ']'); + + topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer)); + + processFailure(topEx, null); + + batches.remove(nodeId); + + if (batches.isEmpty()) // Wait for all pending requests. + onDone(); + + } + + if (log.isDebugEnabled()) + log.debug("Future does not have mapping for left node (ignoring) [nodeId=" + nodeId + + ", fut=" + this + ']'); + + return false; + } + + /** + * @param nodeId Originating node ID. + * @param res Response. + * @param err Exception. + * @return {@code True} if future was completed by this call. + */ + public boolean checkResponse(UUID nodeId, GridNearTxQueryResultsEnlistResponse res, Throwable err) { + assert res != null || err != null : this; + + if (err == null && res.error() != null) + err = res.error(); + + if (X.hasCause(err, ClusterTopologyCheckedException.class)) + tx.removeMapping(nodeId); + + if (err != null) + processFailure(err, null); + + if (ex != null) { + batches.remove(nodeId); + + if (batches.isEmpty()) // Wait for all pending requests. + onDone(); + + return false; + } + + assert res != null; + + RES_UPD.getAndAdd(this, res.result()); + + return true; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxQueryResultsEnlistFuture.class, this, super.toString()); + } + + /** + * A batch of rows + */ + private class Batch { + /** Node ID. */ + @GridToStringExclude + private final ClusterNode node; + + /** Rows. */ + private ArrayList rows = new ArrayList<>(); + + /** Local backup rows. */ + private ArrayList locBkpRows; + + /** Readiness flag. Set when batch is full or no new rows are expected. */ + private boolean ready; + + /** + * @param node Cluster node. + */ + private Batch(ClusterNode node) { + this.node = node; + } + + /** + * @return Node. + */ + public ClusterNode node() { + return node; + } + + /** + * Adds a row. + * + * @param row Row. + * @param localBackup {@code true}, when the row key has local backup. + */ + public void add(Object row, boolean localBackup) { + rows.add(row); + + if (localBackup) { + if (locBkpRows == null) + locBkpRows = new ArrayList<>(); + + locBkpRows.add(row); + } + } + + /** + * @return number of rows. + */ + public int size() { + return rows.size(); + } + + /** + * @return Collection of rows. + */ + public Collection rows() { + return rows; + } + + /** + * @return Collection of local backup rows. + */ + public List localBackupRows() { + return locBkpRows; + } + + /** + * @return Readiness flag. + */ + public boolean ready() { + return ready; + } + + /** + * Sets readiness flag. + * + * @param ready Flag value. + */ + public void ready(boolean ready) { + this.ready = ready; + } + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java new file mode 100644 index 0000000000000..f350d502b9b7d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistRequest.java @@ -0,0 +1,578 @@ +/* + * 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.near; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * Request to enlist into transaction and acquire locks for entries produced + * with complex DML queries with reducer step. + * + * One request per batch of entries is used. + */ +public class GridNearTxQueryResultsEnlistRequest extends GridCacheIdMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long threadId; + + /** */ + private IgniteUuid futId; + + /** */ + private boolean clientFirst; + + /** */ + private int miniId; + + /** */ + private UUID subjId; + + /** */ + private AffinityTopologyVersion topVer; + + /** */ + private GridCacheVersion lockVer; + + /** */ + private MvccSnapshot mvccSnapshot; + + /** */ + private long timeout; + + /** */ + private long txTimeout; + + /** */ + private int taskNameHash; + + /** */ + @GridDirectTransient + private Collection rows; + + /** */ + @GridToStringExclude + private KeyCacheObject[] keys; + + /** */ + @GridToStringExclude + private CacheObject[] values; + + /** */ + private EnlistOperation op; + + /** + * Default constructor. + */ + public GridNearTxQueryResultsEnlistRequest() { + // No-op. + } + + /** + * @param cacheId Cache id. + * @param threadId Thread id. + * @param futId Future id. + * @param miniId Mini-future id. + * @param subjId Transaction subject id. + * @param topVer Topology version. + * @param lockVer Lock version. + * @param mvccSnapshot Mvcc snapshot. + * @param clientFirst First client request flag. + * @param timeout Timeout. + * @param txTimeout Tx timeout. + * @param taskNameHash Task name hash. + * @param rows Rows. + * @param op Operation. + */ + GridNearTxQueryResultsEnlistRequest(int cacheId, + long threadId, + IgniteUuid futId, + int miniId, + UUID subjId, + AffinityTopologyVersion topVer, + GridCacheVersion lockVer, + MvccSnapshot mvccSnapshot, + boolean clientFirst, + long timeout, + long txTimeout, int taskNameHash, + Collection rows, + EnlistOperation op) { + this.txTimeout = txTimeout; + this.cacheId = cacheId; + this.threadId = threadId; + this.futId = futId; + this.miniId = miniId; + this.subjId = subjId; + this.topVer = topVer; + this.lockVer = lockVer; + this.mvccSnapshot = mvccSnapshot; + this.clientFirst = clientFirst; + this.timeout = timeout; + this.taskNameHash = taskNameHash; + this.rows = rows; + this.op = op; + } + + /** + * @return Thread id. + */ + public long threadId() { + return threadId; + } + + /** + * @return Future id. + */ + public IgniteUuid futureId() { + return futId; + } + + /** + * @return Mini future ID. + */ + public int miniId() { + return miniId; + } + + /** + * @return Subject id. + */ + public UUID subjectId() { + return subjId; + } + + /** + * @return Topology version. + */ + @Override public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** + * @return Lock version. + */ + public GridCacheVersion version() { + return lockVer; + } + + /** + * @return MVCC snapshot. + */ + public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + + /** + * @return Timeout milliseconds. + */ + public long timeout() { + return timeout; + } + + /** + * @return Tx timeout milliseconds. + */ + public long txTimeout() { + return txTimeout; + } + + /** + * @return Task name hash. + */ + public int taskNameHash() { + return taskNameHash; + } + + /** + * @return {@code True} if this is the first client request. + */ + public boolean firstClientRequest() { + return clientFirst; + } + + /** + * @return Collection of rows. + */ + public Collection rows() { + return rows; + } + + /** + * @return Operation. + */ + public EnlistOperation operation() { + return op; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + GridCacheContext cctx = ctx.cacheContext(cacheId); + CacheObjectContext objCtx = cctx.cacheObjectContext(); + + if (rows != null && keys == null) { + keys = new KeyCacheObject[rows.size()]; + + int i = 0; + + boolean keysOnly = op.isDeleteOrLock(); + + values = keysOnly ? null : new CacheObject[keys.length]; + + for (Object row : rows) { + Object key, val = null; + + if (keysOnly) + key = row; + else { + key = ((IgniteBiTuple)row).getKey(); + val = ((IgniteBiTuple)row).getValue(); + } + + assert key != null && (keysOnly || val != null): "key=" + key + ", val=" + val; + + KeyCacheObject key0 = cctx.toCacheKeyObject(key); + + assert key0 != null; + + key0.prepareMarshal(objCtx); + + keys[i] = key0; + + if (!keysOnly) { + CacheObject val0 = cctx.toCacheObject(val); + + assert val0 != null; + + val0.prepareMarshal(objCtx); + + values[i] = val0; + } + + i++; + } + } + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + if (keys != null) { + rows = new ArrayList<>(keys.length); + + CacheObjectContext objCtx = ctx.cacheContext(cacheId).cacheObjectContext(); + + for (int i = 0; i < keys.length; i++) { + keys[i].finishUnmarshal(objCtx, ldr); + + if (op.isDeleteOrLock()) + rows.add(keys[i]); + else { + if (values[i] != null) + values[i].finishUnmarshal(objCtx, ldr); + + rows.add(new IgniteBiTuple<>(keys[i], values[i])); + } + } + + keys = null; + values = null; + } + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeBoolean("clientFirst", clientFirst)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeIgniteUuid("futId", futId)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeObjectArray("keys", keys, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeMessage("lockVer", lockVer)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeInt("miniId", miniId)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + return false; + + writer.incrementState(); + + case 9: + if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeUuid("subjId", subjId)) + return false; + + writer.incrementState(); + + case 11: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 12: + if (!writer.writeLong("threadId", threadId)) + return false; + + writer.incrementState(); + + case 13: + if (!writer.writeLong("timeout", timeout)) + return false; + + writer.incrementState(); + + case 14: + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + case 15: + if (!writer.writeLong("txTimeout", txTimeout)) + return false; + + writer.incrementState(); + + case 16: + if (!writer.writeObjectArray("values", values, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + clientFirst = reader.readBoolean("clientFirst"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + futId = reader.readIgniteUuid("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + keys = reader.readObjectArray("keys", MessageCollectionItemType.MSG, KeyCacheObject.class); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + lockVer = reader.readMessage("lockVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + miniId = reader.readInt("miniId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: + mvccSnapshot = reader.readMessage("mvccSnapshot"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 9: + byte opOrd; + + opOrd = reader.readByte("op"); + + if (!reader.isLastRead()) + return false; + + op = EnlistOperation.fromOrdinal(opOrd); + + reader.incrementState(); + + case 10: + subjId = reader.readUuid("subjId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 12: + threadId = reader.readLong("threadId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 13: + timeout = reader.readLong("timeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 14: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 15: + txTimeout = reader.readLong("txTimeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 16: + values = reader.readObjectArray("values", MessageCollectionItemType.MSG, CacheObject.class); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridNearTxQueryResultsEnlistRequest.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 17; + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return false; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 153; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxQueryResultsEnlistRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java new file mode 100644 index 0000000000000..749afd351db75 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistResponse.java @@ -0,0 +1,202 @@ +/* + * 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.near; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * A response to {@link GridNearTxQueryResultsEnlistRequest}. + */ +public class GridNearTxQueryResultsEnlistResponse extends GridNearTxQueryEnlistResponse { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private GridCacheVersion dhtVer; + + /** */ + private IgniteUuid dhtFutId; + + /** */ + private GridLongList updCntrs; + + /** + * Default-constructor. + */ + public GridNearTxQueryResultsEnlistResponse() { + // No-op. + } + + /** + * @param cacheId Cache id. + * @param futId Future id. + * @param miniId Mini future id. + * @param lockVer Lock version. + * @param res Result. + * @param dhtFutId Dht future id. + * @param dhtVer Dht version. + * @param updCntrs Update counters. + */ + public GridNearTxQueryResultsEnlistResponse(int cacheId, + IgniteUuid futId, + int miniId, + GridCacheVersion lockVer, + long res, + GridCacheVersion dhtVer, + IgniteUuid dhtFutId, + GridLongList updCntrs) { + super(cacheId, futId, miniId, lockVer, res, false); + + this.dhtVer = dhtVer; + this.dhtFutId = dhtFutId; + this.updCntrs = updCntrs; + } + + /** + * @param cacheId Cache id. + * @param futId Future id. + * @param miniId Mini future id. + * @param lockVer Lock version. + * @param err Error. + */ + public GridNearTxQueryResultsEnlistResponse(int cacheId, + IgniteUuid futId, + int miniId, + GridCacheVersion lockVer, + Throwable err) { + super(cacheId, futId, miniId, lockVer, err); + } + + /** + * @return Dht version. + */ + public GridCacheVersion dhtVersion() { + return dhtVer; + } + + /** + * @return Dht future id. + */ + public IgniteUuid dhtFutureId() { + return dhtFutId; + } + + /** + * @return Update counters. + */ + public GridLongList updateCounters() { + return updCntrs; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 12; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 9: + if (!writer.writeIgniteUuid("dhtFutId", dhtFutId)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeMessage("dhtVer", dhtVer)) + return false; + + writer.incrementState(); + + case 11: + if (!writer.writeMessage("updCntrs", updCntrs)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 9: + dhtFutId = reader.readIgniteUuid("dhtFutId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: + dhtVer = reader.readMessage("dhtVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: + updCntrs = reader.readMessage("updCntrs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(GridNearTxQueryResultsEnlistResponse.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 154; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxQueryResultsEnlistResponse.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java index 5477af9593786..879bf26fb7273 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java @@ -243,13 +243,16 @@ public GridNearTxRemote( return nearXidVer; } - /** - * @param cntrs Partition indexes. - */ + /** {@inheritDoc} */ @Override public void setPartitionUpdateCounters(long[] cntrs) { // No-op. } + /** {@inheritDoc} */ + @Override public void addActiveCache(GridCacheContext cacheCtx, boolean recovery) throws IgniteCheckedException { + throw new UnsupportedOperationException("Near tx doesn't track active caches."); + } + /** * Adds owned versions to map. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java new file mode 100644 index 0000000000000..a5ab7cd49825d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java @@ -0,0 +1,462 @@ +/* + * 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.near; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheCompoundIdentityFuture; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; +import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +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.lang.IgniteUuid; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * A future tracking requests for remote nodes transaction enlisting and locking + * of entries produced with complex DML queries requiring reduce step. + */ +public class GridNearTxSelectForUpdateFuture extends GridCacheCompoundIdentityFuture + implements GridCacheVersionedFuture { + /** */ + private static final long serialVersionUID = 6931664882548658420L; + + /** Done field updater. */ + private static final AtomicIntegerFieldUpdater DONE_UPD = + AtomicIntegerFieldUpdater.newUpdater(GridNearTxSelectForUpdateFuture.class, "done"); + + /** Done field updater. */ + private static final AtomicReferenceFieldUpdater EX_UPD = + AtomicReferenceFieldUpdater.newUpdater(GridNearTxSelectForUpdateFuture.class, Throwable.class, "ex"); + + /** */ + @SuppressWarnings("unused") + @GridToStringExclude + private volatile int done; + + /** */ + @SuppressWarnings("unused") + @GridToStringExclude + private volatile Throwable ex; + + /** Cache context. */ + @GridToStringExclude + private final GridCacheContext cctx; + + /** Transaction. */ + private final GridNearTxLocal tx; + + /** Mvcc future id. */ + private final IgniteUuid futId; + + /** Lock version. */ + private final GridCacheVersion lockVer; + + /** */ + private AffinityTopologyVersion topVer; + + /** */ + private final long timeout; + + /** Logger. */ + @GridToStringExclude + private final IgniteLogger log; + + /** Timeout object. */ + @GridToStringExclude + private LockTimeoutObject timeoutObj; + + /** Ids of mini futures. */ + private final Map miniFutIds = new HashMap<>(); + + /** + * @param cctx Cache context. + * @param tx Transaction. + * @param timeout Timeout. + */ + public GridNearTxSelectForUpdateFuture(GridCacheContext cctx, + GridNearTxLocal tx, + long timeout) { + super(CU.longReducer()); + + this.cctx = cctx; + this.tx = tx; + this.timeout = timeout; + + futId = IgniteUuid.randomUuid(); + lockVer = tx.xidVersion(); + + log = cctx.logger(GridNearTxSelectForUpdateFuture.class); + } + + /** + * @return Cache context. + */ + public GridCacheContext cache() { + return cctx; + } + + /** + * @param node Node. + */ + private void map(ClusterNode node) { + GridDistributedTxMapping mapping = tx.mappings().get(node.id()); + + if (mapping == null) + tx.mappings().put(mapping = new GridDistributedTxMapping(node)); + + mapping.markQueryUpdate(); + + if (node.isLocal()) + tx.colocatedLocallyMapped(true); + + int futId = futuresCountNoLock(); + + miniFutIds.put(node.id(), futId); + + add(new NodeFuture(node)); + } + + /** + * Process result of query execution on given + * @param nodeId Node id. + * @param cnt Total rows counter on given node. + * @param removeMapping Whether transaction mapping should be removed for node. + * @param err Error. + */ + public void onResult(UUID nodeId, Long cnt, boolean removeMapping, @Nullable Throwable err) { + NodeFuture nodeFut = mapFuture(nodeId); + + if (nodeFut != null) + nodeFut.onResult(cnt, removeMapping, err); + } + + /** {@inheritDoc} */ + @Override protected boolean processFailure(Throwable err, IgniteInternalFuture fut) { + if (ex != null || !EX_UPD.compareAndSet(this, null, err)) + ex.addSuppressed(err); + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean onDone(@Nullable Long res, @Nullable Throwable err, boolean cancelled) { + if (!DONE_UPD.compareAndSet(this, 0, 1)) + return false; + + cctx.tm().txContext(tx); + + Throwable ex0 = ex; + + if (ex0 != null) { + if (err != null) + ex0.addSuppressed(err); + + err = ex0; + } + + if (!cancelled && err == null) + tx.clearLockFuture(this); + else + tx.setRollbackOnly(); + + boolean done = super.onDone(res, err, cancelled); + + assert done; + + // Clean up. + cctx.mvcc().removeVersionedFuture(this); + + if (timeoutObj != null) + cctx.time().removeTimeoutObject(timeoutObj); + + return true; + } + + /** + * Finds pending map node future by the given ID. + * + * @param nodeId Node id. + * @return Map node future. + */ + private NodeFuture mapFuture(UUID nodeId) { + synchronized (this) { + Integer idx = miniFutIds.get(nodeId); + + if (idx == null) + throw new IllegalStateException("SELECT FOR UPDATE node future not found [nodeId=" + nodeId + "]."); + + assert idx >= 0 && idx < futuresCountNoLock(); + + IgniteInternalFuture fut = future(idx); + + if (!fut.isDone()) + return (NodeFuture)fut; + } + + return null; + } + + /** {@inheritDoc} */ + @Override public GridCacheVersion version() { + return lockVer; + } + + /** {@inheritDoc} */ + @Override public boolean onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate owner) { + return false; + } + + /** {@inheritDoc} */ + @Override public IgniteUuid futureId() { + return futId; + } + + /** {@inheritDoc} */ + @Override public boolean onNodeLeft(UUID nodeId) { + if (topVer == null) + return false; // Local query, do nothing. + + for (IgniteInternalFuture fut : futures()) { + NodeFuture f = (NodeFuture)fut; + + if (f.node.id().equals(nodeId)) { + if (log.isDebugEnabled()) + log.debug("Found mini-future for left node [nodeId=" + nodeId + ", mini=" + f + ", fut=" + + this + ']'); + + ClusterTopologyCheckedException topEx = new ClusterTopologyCheckedException("Failed to enlist keys " + + "(primary node left grid, retry transaction if possible) [node=" + nodeId + ']'); + + topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer)); + + return f.onResult(0, false, topEx); + } + } + + if (log.isDebugEnabled()) + log.debug("Future does not have mapping for left node (ignoring) [nodeId=" + nodeId + + ", fut=" + this + ']'); + + return false; + } + + /** {@inheritDoc} */ + @Override public boolean trackable() { + return true; + } + + /** {@inheritDoc} */ + @Override public void markNotTrackable() { + // No-op. + } + + /** {@inheritDoc} */ + @Override protected void logError(IgniteLogger log, String msg, Throwable e) { + // no-op + } + + /** {@inheritDoc} */ + @Override protected void logDebug(IgniteLogger log, String msg) { + // no-op + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearTxSelectForUpdateFuture.class, this, super.toString()); + } + + /** + * Initialize this future for distributed execution. + * @param topVer Topology version. + * @param nodes Nodes to run query on. + */ + public synchronized void init(AffinityTopologyVersion topVer, Collection nodes) { + doInit(topVer, nodes, false); + } + + /** + * Initialize this future for local execution. + */ + public synchronized void initLocal() { + doInit(null, Collections.singletonList(cctx.localNode()), true); + } + + /** + * Initialize this future for distributed or local execution. + * @param topVer Topology version ({@code null} for local case). + * @param nodes Nodes to run query on. + * @param loc Local query flag. + */ + private void doInit(@Nullable AffinityTopologyVersion topVer, Collection nodes, boolean loc) { + assert !loc || (topVer == null && nodes.size() == 1 && nodes.iterator().next().isLocal()); + if (initialized()) + throw new IllegalStateException("SELECT FOR UPDATE future has been initialized already."); + + tx.init(); + + if (timeout < 0) { + // Time is out. + onDone(timeoutException()); + + return; + } + else if (timeout > 0) + timeoutObj = new LockTimeoutObject(); + + if (!tx.updateLockFuture(null, this)) { + onDone(tx.timedOut() ? tx.timeoutException() : tx.rollbackException()); + + return; + } + + boolean added = cctx.mvcc().addFuture(this); + + assert added : this; + + try { + tx.addActiveCache(cctx, false); + } + catch (IgniteCheckedException e) { + onDone(e); + + return; + } + + if (timeoutObj != null) + cctx.time().addTimeoutObject(timeoutObj); + + this.topVer = topVer; + + for (ClusterNode n : nodes) + map(n); + + markInitialized(); + } + + /** + * @return Timeout exception. + */ + @NotNull private IgniteTxTimeoutCheckedException timeoutException() { + return new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " + + "transaction [timeout=" + timeout + ", tx=" + tx + ']'); + } + + /** + * A future tracking a single MAP request to be enlisted in transaction and locked on data node. + */ + private class NodeFuture extends GridFutureAdapter { + /** */ + private boolean completed; + + /** Node ID. */ + @GridToStringExclude + private final ClusterNode node; + + /** + * @param node Cluster node. + * + */ + private NodeFuture(ClusterNode node) { + this.node = node; + } + + /** + * @return Node. + */ + public ClusterNode node() { + return node; + } + + /** + * @param cnt Total rows counter on given node. + * @param removeMapping Whether transaction mapping should be removed for node. + * @param err Exception. + * @return {@code True} if future was completed by this call. + */ + public boolean onResult(long cnt, boolean removeMapping, Throwable err) { + synchronized (this) { + if (completed) + return false; + + completed = true; + } + + if (X.hasCause(err, ClusterTopologyCheckedException.class) || removeMapping) { + GridDistributedTxMapping m = tx.mappings().get(node.id()); + + assert m != null && m.empty(); + + tx.removeMapping(node.id()); + + if (node.isLocal()) + tx.colocatedLocallyMapped(false); + } + else if (err == null && cnt > 0 && !node.isLocal()) + tx.hasRemoteLocks(true); + + return onDone(cnt, err); + } + } + + /** + * Lock request timeout object. + */ + private class LockTimeoutObject extends GridTimeoutObjectAdapter { + /** + * Default constructor. + */ + LockTimeoutObject() { + super(timeout); + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (log.isDebugEnabled()) + log.debug("Timed out waiting for lock response: " + this); + + onDone(timeoutException()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(LockTimeoutObject.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/NearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/NearTxFinishFuture.java index 132c7549657cb..44a87f237efb2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/NearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/NearTxFinishFuture.java @@ -17,15 +17,34 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; /** * */ -public interface NearTxFinishFuture extends IgniteInternalFuture { +public interface NearTxFinishFuture extends IgniteInternalFuture { /** * @return Commit flag. */ boolean commit(); + + /** + * @return Transaction. + */ + GridNearTxLocal tx(); + + /** + * + * @param commit Commit flag. + * @param clearThreadMap If {@code true} removes {@link GridNearTxLocal} from thread map. + * @param onTimeout If {@code true} called from timeout handler. + */ + public void finish(boolean commit, boolean clearThreadMap, boolean onTimeout); + + /** + * @param e Error. + */ + public void onNodeStop(IgniteCheckedException e); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/TxTopologyVersionFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/TxTopologyVersionFuture.java new file mode 100644 index 0000000000000..b5e38839b75b4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/TxTopologyVersionFuture.java @@ -0,0 +1,155 @@ +/* + * 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.near; + +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.CacheStoppedException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.CI1; + +/** + * Future to obtain/lock topology version for SELECT FOR UPDATE. + */ +public class TxTopologyVersionFuture extends GridFutureAdapter { + /** Transaction. */ + private final GridNearTxLocal tx; + + /** Target cache context. */ + private final GridCacheContext cctx; + + /** Topology locked flag. */ + private boolean topLocked; + + /** + * @param tx Transaction. + * @param cctx Target cache context. + */ + public TxTopologyVersionFuture(GridNearTxLocal tx, GridCacheContext cctx) { + this.tx = tx; + this.cctx = cctx; + + init(); + } + + /** */ + private void init() { + // Obtain the topology version to use. + long threadId = Thread.currentThread().getId(); + + AffinityTopologyVersion topVer = cctx.mvcc().lastExplicitLockTopologyVersion(threadId); + + // If there is another system transaction in progress, use it's topology version to prevent deadlock. + if (topVer == null && tx.system()) + topVer = cctx.tm().lockedTopologyVersion(threadId, tx); + + if (topVer != null) + tx.topologyVersion(topVer); + + if (topVer == null) + topVer = tx.topologyVersionSnapshot(); + + if (topVer != null) { + for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()) { + if (fut.exchangeDone() && fut.topologyVersion().equals(topVer)) { + Throwable err = fut.validateCache(cctx, false, false, null, null); + + if (err != null) { + onDone(err); + + return; + } + + break; + } + } + + onDone(topVer); + + topLocked = true; + + return; + } + + acquireTopologyVersion(); + } + + /** + * Acquire topology future and wait for its completion. + */ + private void acquireTopologyVersion() { + cctx.topology().readLock(); + + try { + if (cctx.topology().stopping()) { + onDone(new CacheStoppedException(cctx.name())); + + return; + } + + GridDhtTopologyFuture fut = cctx.topologyVersionFuture(); + + if (fut.isDone()) { + Throwable err = fut.validateCache(cctx, false, false, null, null); + + if (err != null) { + onDone(err); + + return; + } + + AffinityTopologyVersion topVer = fut.topologyVersion(); + + if (tx != null) + tx.topologyVersion(topVer); + + onDone(topVer); + } + else { + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + fut.get(); + + acquireTopologyVersion(); + } + catch (IgniteCheckedException e) { + onDone(e); + } + finally { + cctx.shared().txContextReset(); + } + } + }); + } + } + finally { + cctx.topology().readUnlock(); + } + } + + /** + * @return Client first flag. + */ + public boolean clientFirst() { + return cctx.localNode().isClient() && !topLocked && !tx.hasRemoteLocks(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java index f2a4b30c4af82..f36237491f18f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheManager; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.dr.GridDrType; import org.jetbrains.annotations.Nullable; @@ -55,6 +56,36 @@ public void replicate(KeyCacheObject key, GridDrType drType, AffinityTopologyVersion topVer)throws IgniteCheckedException; + /** + * Enlist for DR. + * + * @param key Key. + * @param val Value. + * @param ttl TTL. + * @param expireTime Expire time. + * @param ver Version. + * @param drType Replication type. + * @param topVer Topology version. + * @param mvccVer Tx mvcc version. + * @throws IgniteCheckedException If failed. + */ + void mvccReplicate(KeyCacheObject key, + @Nullable CacheObject val, + long ttl, + long expireTime, + GridCacheVersion ver, + GridDrType drType, + AffinityTopologyVersion topVer, + MvccVersion mvccVer) throws IgniteCheckedException; + + /** + * @param mvccVer Tx mvcc version. + * @param commit {@code True} if tx committed, {@code False} otherwise. + * @param topVer Tx snapshot affinity version. + * @throws IgniteCheckedException If failed. + */ + void onTxFinished(MvccVersion mvccVer, boolean commit, AffinityTopologyVersion topVer); + /** * Process partitions exchange event. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java index f3c1b23f7c7d6..8d7e4d8bedd0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.lang.IgniteFuture; @@ -77,6 +78,18 @@ public class GridOsCacheDrManager implements GridCacheDrManager { // No-op. } + /** {@inheritDoc} */ + @Override public void mvccReplicate(KeyCacheObject key, @Nullable CacheObject val, long ttl, long expireTime, + GridCacheVersion ver, GridDrType drType, AffinityTopologyVersion topVer, + MvccVersion mvccVer) throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onTxFinished(MvccVersion mvccVer, boolean commit, AffinityTopologyVersion topVer) { + // No-op. + } + /** {@inheritDoc} */ @Override public void onExchange(AffinityTopologyVersion topVer, boolean left) throws IgniteCheckedException { // No-op. 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 ef64ed8dad2a6..b615952f785cd 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 @@ -54,8 +54,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.local.GridLocalCache; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.resource.GridResourceIoc; @@ -461,6 +461,7 @@ private Map getAllInternal(@Nullable Collection keys, taskName, expiry, !deserializeBinary, + null, null); if (res != null) { @@ -488,7 +489,8 @@ private Map getAllInternal(@Nullable Collection keys, null, taskName, expiry, - !deserializeBinary); + !deserializeBinary, + null); if (v != null) { ctx.addResult(vals, @@ -1096,7 +1098,8 @@ private Map updateWithBatch( entryProcessor, taskName, null, - keepBinary); + keepBinary, + null); Object oldVal = null; @@ -1238,7 +1241,8 @@ else if (op == UPDATE) { null, taskName, null, - keepBinary); + keepBinary, + null); Object interceptorVal = ctx.config().getInterceptor().onBeforePut(new CacheLazyEntry( ctx, entry.key(), old, keepBinary), val); @@ -1273,7 +1277,8 @@ else if (op == UPDATE) { null, taskName, null, - keepBinary); + keepBinary, + null); IgniteBiTuple interceptorRes = ctx.config().getInterceptor() .onBeforeRemove(new CacheLazyEntry(ctx, entry.key(), old, keepBinary)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCoordinator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCoordinator.java new file mode 100644 index 0000000000000..045177ad7ec6a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCoordinator.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.internal.processors.cache.mvcc; + +import java.io.Serializable; +import java.util.UUID; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +public class MvccCoordinator implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final UUID nodeId; + + /** + * Unique coordinator version, increases when new coordinator is assigned, + * can differ from topVer if we decide to assign coordinator manually. + */ + private final long crdVer; + + /** */ + @GridToStringInclude + private final AffinityTopologyVersion topVer; + + /** + * @param nodeId Coordinator node ID. + * @param crdVer Coordinator version. + * @param topVer Topology version when coordinator was assigned. + */ + public MvccCoordinator(UUID nodeId, long crdVer, AffinityTopologyVersion topVer) { + assert nodeId != null; + assert crdVer > 0 : crdVer; + assert topVer != null; + + this.nodeId = nodeId; + this.crdVer = crdVer; + this.topVer = topVer; + } + + /** + * @return Unique coordinator version. + */ + public long coordinatorVersion() { + return crdVer; + } + + /** + * @return Coordinator node ID. + */ + public UUID nodeId() { + return nodeId; + } + + /** + * @return Topology version when coordinator was assigned. + */ + public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + MvccCoordinator that = (MvccCoordinator)o; + + return crdVer == that.crdVer; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return (int)(crdVer ^ (crdVer >>> 32)); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccCoordinator.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccDiscoveryData.java new file mode 100644 index 0000000000000..d2e936fd346a5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccDiscoveryData.java @@ -0,0 +1,52 @@ +/* + * 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.mvcc; + +import org.apache.ignite.internal.util.typedef.internal.S; + +import java.io.Serializable; + +/** + * MVCC discovery data to be shared between nodes on join. + */ +public class MvccDiscoveryData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Current coordinator. */ + private MvccCoordinator crd; + + /** + * @param crd Coordinator. + */ + public MvccDiscoveryData(MvccCoordinator crd) { + this.crd = crd; + } + + /** + * @return Current coordinator. + */ + public MvccCoordinator coordinator() { + return crd; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccDiscoveryData.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccEmptyLongList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccEmptyLongList.java new file mode 100644 index 0000000000000..7963685ec6d84 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccEmptyLongList.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.cache.mvcc; + +/** + * + */ +public class MvccEmptyLongList implements MvccLongList { + /** */ + public static MvccEmptyLongList INSTANCE = new MvccEmptyLongList(); + + /** + * + */ + private MvccEmptyLongList() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public int size() { + return 0; + } + + /** {@inheritDoc} */ + @Override public long get(int i) { + throw new IndexOutOfBoundsException(); + } + + /** {@inheritDoc} */ + @Override public boolean contains(long val) { + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "MvccEmptyLongList[]"; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccFuture.java new file mode 100644 index 0000000000000..0ca82d3a47fe6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccFuture.java @@ -0,0 +1,66 @@ +/* + * 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.mvcc; + +import java.util.UUID; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +public class MvccFuture extends GridFutureAdapter { + /** */ + protected UUID crdId; + + /** + * Default constructor. + */ + public MvccFuture() { + } + + /** + * @param crdId MVCC coordinator node ID. + */ + public MvccFuture(UUID crdId) { + assert crdId != null; + + this.crdId = crdId; + } + + /** + * @return MVCC coordinator node ID. + */ + public UUID coordinatorNodeId() { + return crdId; + } + + /** + * @param crdId MVCC coordinator node ID. + */ + public void coordinatorNodeId(UUID crdId) { + assert crdId != null; + + this.crdId = crdId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccFuture.class, this, super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccLongList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccLongList.java new file mode 100644 index 0000000000000..8b580edff2264 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccLongList.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.internal.processors.cache.mvcc; + +/** + * + */ +public interface MvccLongList { + public int size(); + + public long get(int i); + + public boolean contains(long val); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccPreviousCoordinatorQueries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccPreviousCoordinatorQueries.java new file mode 100644 index 0000000000000..024ef70a3c4b0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccPreviousCoordinatorQueries.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.typedef.F; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker.MVCC_TRACKER_ID_NA; + +/** + * + */ +class MvccPreviousCoordinatorQueries { + /** */ + private volatile boolean prevQueriesDone; + + /** Map of nodes to active {@link MvccQueryTracker} IDs list. */ + private final ConcurrentHashMap> activeQueries = new ConcurrentHashMap<>(); + + /** */ + private final ConcurrentHashMap> rcvdAcks = new ConcurrentHashMap<>(); + + /** */ + private Set rcvd; + + /** */ + private Set waitNodes; + + /** */ + private boolean initDone; + + /** + * @param nodeQueries Active queries map. + * @param discoCache Discovery data. + * @param mgr Discovery manager. + */ + void init(Map nodeQueries, DiscoCache discoCache, GridDiscoveryManager mgr) { + synchronized (this) { + assert !initDone; + assert waitNodes == null; + + waitNodes = new HashSet<>(); + + for (ClusterNode node : discoCache.allNodes()) { + if ((nodeQueries == null || !nodeQueries.containsKey(node.id())) && + mgr.alive(node) && + !F.contains(rcvd, node.id())) + waitNodes.add(node.id()); + } + + initDone = waitNodes.isEmpty(); + + if (nodeQueries != null) { + for (Map.Entry e : nodeQueries.entrySet()) + mergeToActiveQueries(e.getKey(), e.getValue()); + } + + if (initDone && !prevQueriesDone) + prevQueriesDone = activeQueries.isEmpty() && rcvdAcks.isEmpty(); + } + } + + /** + * @return {@code True} if all queries on + */ + boolean previousQueriesDone() { + return prevQueriesDone; + } + + /** + * Merges current node active queries with the given ones. + * + * @param nodeId Node ID. + * @param nodeTrackers Active query trackers started on node. + */ + private void mergeToActiveQueries(UUID nodeId, GridLongList nodeTrackers) { + if (nodeTrackers == null || nodeTrackers.isEmpty() || prevQueriesDone) + return; + + Set currTrackers = activeQueries.get(nodeId); + + if (currTrackers == null) + activeQueries.put(nodeId, currTrackers = addAll(nodeTrackers, null)); + else + addAll(nodeTrackers, currTrackers); + + // Check if there were any acks had been arrived before. + Set currAcks = rcvdAcks.get(nodeId); + + if (!currTrackers.isEmpty() && currAcks != null && !currAcks.isEmpty()) { + Collection intersection = new HashSet<>(currAcks); + + intersection.retainAll(currTrackers); + + currAcks.removeAll(intersection); + currTrackers.removeAll(intersection); + + if (currTrackers.isEmpty()) + activeQueries.remove(nodeId); + + if (currAcks.isEmpty()) + rcvdAcks.remove(nodeId); + } + + if (initDone && !prevQueriesDone) + prevQueriesDone = activeQueries.isEmpty() && rcvdAcks.isEmpty(); + } + + /** + * @param nodeId Node ID. + * @param nodeTrackers Active query trackers started on node. + */ + void addNodeActiveQueries(UUID nodeId, @Nullable GridLongList nodeTrackers) { + synchronized (this) { + if (initDone) + return; + + if (waitNodes == null) { + if (rcvd == null) + rcvd = new HashSet<>(); + + rcvd.add(nodeId); + } + else { + waitNodes.remove(nodeId); + + initDone = waitNodes.isEmpty(); + } + + mergeToActiveQueries(nodeId, nodeTrackers); + + if (initDone && !prevQueriesDone) + prevQueriesDone = activeQueries.isEmpty() && rcvdAcks.isEmpty(); + } + } + + /** + * @param nodeId Failed node ID. + */ + void onNodeFailed(UUID nodeId) { + synchronized (this) { + if (waitNodes != null) { + waitNodes.remove(nodeId); + + initDone = waitNodes.isEmpty(); + } + + if (initDone && !prevQueriesDone && activeQueries.remove(nodeId) != null) + prevQueriesDone = activeQueries.isEmpty() && rcvdAcks.isEmpty(); + } + } + + /** + * @param nodeId Node ID. + * @param qryTrackerId Query tracker Id. + */ + void onQueryDone(UUID nodeId, long qryTrackerId) { + if (qryTrackerId == MVCC_TRACKER_ID_NA) + return; + + synchronized (this) { + Set nodeTrackers = activeQueries.get(nodeId); + + if (nodeTrackers == null || !nodeTrackers.remove(qryTrackerId)) { + Set nodeAcks = rcvdAcks.get(nodeId); + + if (nodeAcks == null) + rcvdAcks.put(nodeId, nodeAcks = new HashSet<>()); + + // We received qry done ack before the active qry message. Need to save it. + nodeAcks.add(qryTrackerId); + } + + if (nodeTrackers != null && nodeTrackers.isEmpty()) + activeQueries.remove(nodeId); + + if (initDone && !prevQueriesDone) + prevQueriesDone = activeQueries.isEmpty() && rcvdAcks.isEmpty(); + } + } + + /** + * @param from Long list. + * @param to Set. + */ + private Set addAll(GridLongList from, Set to) { + assert from != null; + + if (to == null) + to = new HashSet<>(from.size()); + + for (int i = 0; i < from.size(); i++) + to.add(from.get(i)); + + return to; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java new file mode 100644 index 0000000000000..bce61a18a00de --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import java.util.Collection; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteDiagnosticPrepareContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.processors.GridProcessor; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.ExchangeContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.util.GridLongList; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public interface MvccProcessor extends GridProcessor { + /** + * @param evtType Event type. + * @param nodes Current nodes. + * @param topVer Topology version. + */ + void onDiscoveryEvent(int evtType, Collection nodes, long topVer); + + /** + * Exchange start callback. + * + * @param mvccCrd Mvcc coordinator. + * @param exchCtx Exchange context. + * @param exchCrd Exchange coordinator. + */ + void onExchangeStart(MvccCoordinator mvccCrd, ExchangeContext exchCtx, ClusterNode exchCrd); + + /** + * Exchange done callback. + * + * @param newCoord New coordinator flag. + * @param discoCache Disco cache. + * @param activeQueries Active queries. + */ + void onExchangeDone(boolean newCoord, DiscoCache discoCache, Map activeQueries); + + /** + * @param nodeId Node ID + * @param activeQueries Active queries. + */ + void processClientActiveQueries(UUID nodeId, @Nullable GridLongList activeQueries); + + /** + * @return Coordinator. + */ + @Nullable MvccCoordinator currentCoordinator(); + + /** + * Check that the given topology is greater or equals to coordinator's one and returns current coordinator. + * @param topVer Topology version. + * @return Mvcc coordinator. + */ + @Nullable MvccCoordinator currentCoordinator(AffinityTopologyVersion topVer); + + /** + * @return Mvcc coordinator received from discovery event. + */ + @Nullable MvccCoordinator coordinatorFromDiscoveryEvent(); + + /** + * @return Current coordinator node ID. + */ + UUID currentCoordinatorId(); + + /** + * @param curCrd Coordinator. + */ + void updateCoordinator(MvccCoordinator curCrd); + + /** + * @param crdVer Mvcc coordinator version. + * @param cntr Mvcc counter. + * @return State for given mvcc version. + * @throws IgniteCheckedException If fails. + */ + byte state(long crdVer, long cntr) throws IgniteCheckedException; + + /** + * @param ver Version to check. + * @return State for given mvcc version. + * @throws IgniteCheckedException If fails. + */ + byte state(MvccVersion ver) throws IgniteCheckedException; + + /** + * @param ver Version. + * @param state State. + * @throws IgniteCheckedException If fails; + */ + void updateState(MvccVersion ver, byte state) throws IgniteCheckedException; + + /** + * @param ver Version. + * @param state State. + * @param primary Flag if this is primary node. + * @throws IgniteCheckedException If fails; + */ + void updateState(MvccVersion ver, byte state, boolean primary) throws IgniteCheckedException; + + /** + * @param crd Mvcc coordinator version. + * @param cntr Mvcc counter. + */ + void registerLocalTransaction(long crd, long cntr); + + /** + * @param crd Mvcc coordinator version. + * @param cntr Mvcc counter. + * @return {@code True} if there is an active local transaction with given version. + */ + boolean hasLocalTransaction(long crd, long cntr); + + /** + * @param cctx Cache context. + * @param locked Version the entry is locked by. + * @return Future, which is completed as soon as the lock is released. + * @throws IgniteCheckedException If failed. + */ + IgniteInternalFuture waitFor(GridCacheContext cctx, MvccVersion locked) throws IgniteCheckedException; + + /** + * @param tracker Query tracker. + */ + void addQueryTracker(MvccQueryTracker tracker); + + /** + * @param id Query tracker id. + */ + void removeQueryTracker(Long id); + + /** + * @return {@link MvccSnapshot} if this is a coordinator node and coordinator is initialized. + * {@code Null} in other cases. + * @throws ClusterTopologyCheckedException If coordinator doesn't match locked topology or not assigned. + */ + MvccSnapshot tryRequestSnapshotLocal() throws ClusterTopologyCheckedException; + + /** + * @param tx Transaction. + * @return {@link MvccSnapshot} if this is a coordinator node and coordinator is initialized. + * {@code Null} in other cases. + * @throws ClusterTopologyCheckedException If coordinator doesn't match locked topology or not assigned. + */ + MvccSnapshot tryRequestSnapshotLocal(@Nullable IgniteInternalTx tx) throws ClusterTopologyCheckedException; + + /** + * Requests snapshot on Mvcc coordinator. + * + * @return Snapshot future. + */ + IgniteInternalFuture requestSnapshotAsync(); + + /** + * Requests snapshot on Mvcc coordinator. + * + * @param tx Transaction. + * @return Snapshot future. + */ + IgniteInternalFuture requestSnapshotAsync(IgniteInternalTx tx); + + /** + * Requests snapshot on Mvcc coordinator. + * + * @param lsnr Request listener. + */ + void requestSnapshotAsync(MvccSnapshotResponseListener lsnr); + + /** + * Requests snapshot on Mvcc coordinator. + * + * @param tx Transaction + * @param lsnr Request listener. + */ + void requestSnapshotAsync(IgniteInternalTx tx, MvccSnapshotResponseListener lsnr); + + /** + * @param updateVer Transaction update version. + * @return Acknowledge future. + */ + IgniteInternalFuture ackTxCommit(MvccSnapshot updateVer); + + /** + * @param updateVer Transaction update version. + * @param readSnapshot Transaction read version. + * @param qryId Query tracker id. + * @return Acknowledge future. + */ + IgniteInternalFuture ackTxCommit(MvccVersion updateVer, MvccSnapshot readSnapshot, long qryId); + + /** + * @param updateVer Transaction update version. + */ + void ackTxRollback(MvccVersion updateVer); + + /** + * @param updateVer Transaction update version. + * @param readSnapshot Transaction read version. + * @param qryTrackerId Query tracker id. + */ + void ackTxRollback(MvccVersion updateVer, MvccSnapshot readSnapshot, long qryTrackerId); + + /** + * @param snapshot Query version. + * @param qryId Query tracker ID. + */ + void ackQueryDone(MvccSnapshot snapshot, long qryId); + + /** + * @param crdId Coordinator ID. + * @param txs Transaction IDs. + * @return Future. + */ + IgniteInternalFuture waitTxsFuture(UUID crdId, GridLongList txs); + + /** + * @param log Logger. + * @param diagCtx Diagnostic request. + */ + void dumpDebugInfo(IgniteLogger log, @Nullable IgniteDiagnosticPrepareContext diagCtx); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java new file mode 100644 index 0000000000000..31d3b6115bc45 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java @@ -0,0 +1,2187 @@ +/* + * 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.mvcc; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteDiagnosticPrepareContext; +import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.communication.GridMessageListener; +import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.ExchangeContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestQueryCntr; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestQueryId; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTx; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTxAndQueryCntr; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTxAndQueryId; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccActiveQueriesMessage; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccFutureResponse; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccMessage; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccQuerySnapshotRequest; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccSnapshotResponse; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccTxSnapshotRequest; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccWaitTxsRequest; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxKey; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.DatabaseLifecycleListener; +import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.util.GridAtomicLong; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.F; +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.internal.util.worker.GridWorker; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteNodeValidationResult; +import org.apache.ignite.spi.discovery.DiscoveryDataBag; +import org.apache.ignite.thread.IgniteThread; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; +import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; +import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED; +import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED; +import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE_COORDINATOR; +import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; +import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker.MVCC_TRACKER_ID_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_INITIAL_CNTR; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_READ_OP_CNTR; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_START_CNTR; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_START_OP_CNTR; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.compare; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.hasNewVersion; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.isVisible; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.noCoordinatorError; +import static org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog.TX_LOG_CACHE_ID; +import static org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog.TX_LOG_CACHE_NAME; +import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.KEY_ONLY; + +/** + * MVCC processor. + */ +@SuppressWarnings("serial") +class MvccProcessorImpl extends GridProcessorAdapter implements MvccProcessor, DatabaseLifecycleListener { + /** */ + private static final Waiter LOCAL_TRANSACTION_MARKER = new LocalTransactionMarker(); + + /** Dummy tx for vacuum. */ + private static final IgniteInternalTx DUMMY_TX = new GridNearTxLocal(); + + /** For tests only. */ + private static IgniteClosure, ClusterNode> crdC; + + /** + * For testing only. + * + * @param crdC Closure assigning coordinator. + */ + static void coordinatorAssignClosure(IgniteClosure, ClusterNode> crdC) { + MvccProcessorImpl.crdC = crdC; + } + + /** Topology version when local node was assigned as coordinator. */ + private long crdVer; + + /** */ + private volatile MvccCoordinator curCrd; + + /** */ + private TxLog txLog; + + /** */ + private List vacuumWorkers; + + /** */ + private BlockingQueue cleanupQueue; + + /** + * Vacuum mutex. Prevents concurrent vacuum while start/stop operations + */ + private final Object mux = new Object(); + + /** For tests only. */ + private volatile Throwable vacuumError; + + /** */ + private MvccDiscoveryData discoData = new MvccDiscoveryData(null); + + /** */ + private final GridAtomicLong futIdCntr = new GridAtomicLong(0); + + /** */ + private final GridAtomicLong mvccCntr = new GridAtomicLong(MVCC_START_CNTR); + + /** */ + private final GridAtomicLong committedCntr = new GridAtomicLong(MVCC_INITIAL_CNTR); + + /** */ + private final Map activeTxs = new HashMap<>(); + + /** Active query trackers. */ + private final Map activeTrackers = new ConcurrentHashMap<>(); + + /** */ + private final Map> snapLsnrs = new ConcurrentHashMap<>(); + + /** */ + private final Map ackFuts = new ConcurrentHashMap<>(); + + /** */ + private final Map waitTxFuts = new ConcurrentHashMap<>(); + + /** */ + private final Map waitMap = new ConcurrentHashMap<>(); + + /** */ + private final ActiveQueries activeQueries = new ActiveQueries(); + + /** */ + private final MvccPreviousCoordinatorQueries prevCrdQueries = new MvccPreviousCoordinatorQueries(); + + /** */ + private final GridFutureAdapter initFut = new GridFutureAdapter<>(); + + /** + * @param ctx Context. + */ + public MvccProcessorImpl(GridKernalContext ctx) { + super(ctx); + + ctx.internalSubscriptionProcessor().registerDatabaseListener(this); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + ctx.addNodeAttribute(IgniteNodeAttributes.ATTR_MVCC_ENABLED, Boolean.TRUE); + + ctx.event().addLocalEventListener(new CacheCoordinatorNodeFailListener(), + EVT_NODE_FAILED, EVT_NODE_LEFT); + + ctx.io().addMessageListener(TOPIC_CACHE_COORDINATOR, new CoordinatorMessageListener()); + } + + /** {@inheritDoc} */ + @Nullable @Override public IgniteNodeValidationResult validateNode(ClusterNode node) { + Boolean rmtEnabled = node.attribute(IgniteNodeAttributes.ATTR_MVCC_ENABLED); + + if (rmtEnabled == null) { + String errMsg = "Failed to add node to topology because MVCC is enabled on cluster and " + + "the node doesn't support MVCC or MVCC is disabled for the node [nodeId=" + node.id() + ']'; + + return new IgniteNodeValidationResult(node.id(), errMsg, errMsg); + } + + return null; + } + + /** {@inheritDoc} */ + @Override public void beforeStop(IgniteCacheDatabaseSharedManager mgr) { + stopVacuumWorkers(); + + txLog = null; + } + + /** {@inheritDoc} */ + @Override public void onInitDataRegions(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + DataStorageConfiguration dscfg = dataStorageConfiguration(); + + mgr.addDataRegion( + dscfg, + createTxLogRegion(dscfg), + CU.isPersistenceEnabled(ctx.config())); + } + + /** {@inheritDoc} */ + @Override public void afterInitialise(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + if (!CU.isPersistenceEnabled(ctx.config())) { + assert txLog == null; + + txLog = new TxLog(ctx, mgr); + + startVacuumWorkers(); + } + } + + /** {@inheritDoc} */ + @SuppressWarnings("ConstantConditions") + @Override public void beforeMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + assert CU.isPersistenceEnabled(ctx.config()); + assert txLog == null; + + ctx.cache().context().pageStore().initialize(TX_LOG_CACHE_ID, 1, + TX_LOG_CACHE_NAME, mgr.dataRegion(TX_LOG_CACHE_NAME).memoryMetrics()); + } + + /** {@inheritDoc} */ + @Override public void afterMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + assert CU.isPersistenceEnabled(ctx.config()); + assert txLog == null; + + txLog = new TxLog(ctx, mgr); + + startVacuumWorkers(); + } + + /** {@inheritDoc} */ + @Override public DiscoveryDataExchangeType discoveryDataType() { + return DiscoveryDataExchangeType.CACHE_CRD_PROC; + } + + /** {@inheritDoc} */ + @SuppressWarnings("ConstantConditions") + @Override public void collectGridNodeData(DiscoveryDataBag dataBag) { + Integer cmpId = discoveryDataType().ordinal(); + + if (!dataBag.commonDataCollectedFor(cmpId)) + dataBag.addGridCommonData(cmpId, discoData); + } + + /** {@inheritDoc} */ + @Override public void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) { + MvccDiscoveryData discoData0 = (MvccDiscoveryData)data.commonData(); + + // Disco data might be null in case the first joined node is daemon. + if (discoData0 != null) { + discoData = discoData0; + + log.info("Received mvcc coordinator on node join: " + discoData.coordinator()); + + assert discoData != null; + } + } + + /** {@inheritDoc} */ + @Override public void onDiscoveryEvent(int evtType, Collection nodes, long topVer) { + if (evtType == EVT_NODE_METRICS_UPDATED || evtType == EVT_DISCOVERY_CUSTOM_EVT) + return; + + MvccCoordinator crd; + + if (evtType == EVT_NODE_SEGMENTED || evtType == EVT_CLIENT_NODE_DISCONNECTED) + crd = null; + else { + crd = discoData.coordinator(); + + if (crd == null || + ((evtType == EVT_NODE_FAILED || evtType == EVT_NODE_LEFT) && !F.nodeIds(nodes).contains(crd.nodeId()))) { + ClusterNode crdNode = null; + + if (crdC != null) { + crdNode = crdC.apply(nodes); + + if (log.isInfoEnabled()) + log.info("Assigned coordinator using test closure: " + crd); + } + else { + // Expect nodes are sorted by order. + for (ClusterNode node : nodes) { + if (!node.isClient()) { + crdNode = node; + + break; + } + } + } + + crd = crdNode != null ? new MvccCoordinator(crdNode.id(), coordinatorVersion(topVer), new AffinityTopologyVersion(topVer, 0)) : null; + + if (crd != null) { + if (log.isInfoEnabled()) + log.info("Assigned mvcc coordinator [crd=" + crd + ", crdNode=" + crdNode + ']'); + } + else + U.warn(log, "New mvcc coordinator was not assigned [topVer=" + topVer + ']'); + } + } + + discoData = new MvccDiscoveryData(crd); + } + + /** {@inheritDoc} */ + @Override public void onExchangeStart(MvccCoordinator mvccCrd, ExchangeContext exchCtx, ClusterNode exchCrd) { + if (!exchCtx.newMvccCoordinator()) + return; + + GridLongList activeQryTrackers = collectActiveQueryTrackers(); + + exchCtx.addActiveQueries(ctx.localNodeId(), activeQryTrackers); + + if (exchCrd == null || !mvccCrd.nodeId().equals(exchCrd.id())) { + try { + sendMessage(mvccCrd.nodeId(), new MvccActiveQueriesMessage(activeQryTrackers)); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send active queries to mvcc coordinator: " + e); + } + } + } + + /** {@inheritDoc} */ + @Override public void onExchangeDone(boolean newCoord, DiscoCache discoCache, Map activeQueries) { + if (!newCoord) + return; + + ctx.cache().context().tm().rollbackMvccTxOnCoordinatorChange(); + + if (ctx.localNodeId().equals(curCrd.nodeId())) { + assert ctx.localNodeId().equals(curCrd.nodeId()); + + MvccCoordinator crd = discoCache.mvccCoordinator(); + + assert crd != null; + + // No need to re-initialize if coordinator version hasn't changed (e.g. it was cluster activation). + if (crdVer == crd.coordinatorVersion()) + return; + + crdVer = crd.coordinatorVersion(); + + log.info("Initialize local node as mvcc coordinator [node=" + ctx.localNodeId() + + ", crdVer=" + crdVer + ']'); + + prevCrdQueries.init(activeQueries, discoCache, ctx.discovery()); + + initFut.onDone(); + } + } + + /** {@inheritDoc} */ + @Override public void processClientActiveQueries(UUID nodeId, @Nullable GridLongList activeQueries) { + prevCrdQueries.addNodeActiveQueries(nodeId, activeQueries); + } + + /** {@inheritDoc} */ + @Override @Nullable public MvccCoordinator currentCoordinator() { + return currentCoordinator(AffinityTopologyVersion.NONE); + } + + /** {@inheritDoc} */ + @Override @Nullable public MvccCoordinator currentCoordinator(AffinityTopologyVersion topVer) { + MvccCoordinator crd = curCrd; + + // Assert coordinator did not already change. + assert crd == null + || topVer == AffinityTopologyVersion.NONE + || crd.topologyVersion().compareTo(topVer) <= 0 : "Invalid coordinator [crd=" + crd + ", topVer=" + topVer + ']'; + + return crd; + } + + /** {@inheritDoc} */ + @Override @Nullable public MvccCoordinator coordinatorFromDiscoveryEvent() { + return discoData != null ? discoData.coordinator() : null; + } + + /** {@inheritDoc} */ + @Override public UUID currentCoordinatorId() { + MvccCoordinator curCrd = this.curCrd; + + return curCrd != null ? curCrd.nodeId() : null; + } + + /** {@inheritDoc} */ + @Override public void updateCoordinator(MvccCoordinator curCrd) { + this.curCrd = curCrd; + } + + /** {@inheritDoc} */ + @Override public byte state(long crdVer, long cntr) throws IgniteCheckedException { + return txLog.get(crdVer, cntr); + } + + /** {@inheritDoc} */ + @Override public byte state(MvccVersion ver) throws IgniteCheckedException { + return txLog.get(ver.coordinatorVersion(), ver.counter()); + } + + /** {@inheritDoc} */ + @Override public void updateState(MvccVersion ver, byte state) throws IgniteCheckedException { + updateState(ver, state, true); + } + + /** {@inheritDoc} */ + @Override public void updateState(MvccVersion ver, byte state, boolean primary) throws IgniteCheckedException { + TxKey key = new TxKey(ver.coordinatorVersion(), ver.counter()); + + txLog.put(key, state, primary); + + Waiter waiter; + + if (primary && (state == TxState.ABORTED || state == TxState.COMMITTED) + && (waiter = waitMap.remove(key)) != null) + waiter.run(ctx); + } + + /** {@inheritDoc} */ + @Override public void registerLocalTransaction(long crd, long cntr) { + Waiter old = waitMap.putIfAbsent(new TxKey(crd, cntr), LOCAL_TRANSACTION_MARKER); + + assert old == null || old.hasLocalTransaction(); + } + + /** {@inheritDoc} */ + @Override public boolean hasLocalTransaction(long crd, long cntr) { + Waiter waiter = waitMap.get(new TxKey(crd, cntr)); + + return waiter != null && waiter.hasLocalTransaction(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture waitFor(GridCacheContext cctx, MvccVersion locked) throws IgniteCheckedException { + TxKey key = new TxKey(locked.coordinatorVersion(), locked.counter()); + + LockFuture fut = new LockFuture(cctx.ioPolicy()); + + Waiter waiter = waitMap.merge(key, fut, Waiter::concat); + + byte state = txLog.get(key); + + if ((state == TxState.ABORTED || state == TxState.COMMITTED) + && !waiter.hasLocalTransaction() && (waiter = waitMap.remove(key)) != null) + waiter.run(ctx); + + return fut; + } + + /** {@inheritDoc} */ + @Override public void addQueryTracker(MvccQueryTracker tracker) { + assert tracker.id() != MVCC_TRACKER_ID_NA; + + MvccQueryTracker tr = activeTrackers.put(tracker.id(), tracker); + + assert tr == null; + } + + /** {@inheritDoc} */ + @Override public void removeQueryTracker(Long id) { + activeTrackers.remove(id); + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot tryRequestSnapshotLocal() throws ClusterTopologyCheckedException { + return tryRequestSnapshotLocal(null); + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot tryRequestSnapshotLocal(@Nullable IgniteInternalTx tx) throws ClusterTopologyCheckedException { + MvccCoordinator crd = currentCoordinator(); + + if (crd == null) + throw noCoordinatorError(); + + if (tx != null) { + AffinityTopologyVersion topVer = ctx.cache().context().lockedTopologyVersion(null); + + if (topVer != null && topVer.compareTo(crd.topologyVersion()) < 0) + throw new ClusterTopologyCheckedException("Mvcc coordinator is outdated " + + "for the locked topology version. [crd=" + crd + ", tx=" + tx + ']'); + } + + if (!ctx.localNodeId().equals(crd.nodeId()) || !initFut.isDone()) + return null; + else if (tx != null) + return assignTxSnapshot(0L); + else + return activeQueries.assignQueryCounter(ctx.localNodeId(), 0L); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshotAsync() { + return requestSnapshotAsync((IgniteInternalTx)null); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshotAsync(IgniteInternalTx tx) { + MvccSnapshotFuture fut = new MvccSnapshotFuture(); + + requestSnapshotAsync(tx, fut); + + return fut; + } + + /** {@inheritDoc} */ + @Override public void requestSnapshotAsync(MvccSnapshotResponseListener lsnr) { + requestSnapshotAsync(null, lsnr); + } + + /** {@inheritDoc} */ + @Override public void requestSnapshotAsync(IgniteInternalTx tx, MvccSnapshotResponseListener lsnr) { + MvccCoordinator crd = currentCoordinator(); + + if (crd == null) { + lsnr.onError(noCoordinatorError()); + + return; + } + + if (tx != null) { + AffinityTopologyVersion topVer = ctx.cache().context().lockedTopologyVersion(null); + + if (topVer != null && topVer.compareTo(crd.topologyVersion()) < 0) { + lsnr.onError(new ClusterTopologyCheckedException("Mvcc coordinator is outdated " + + "for the locked topology version. [crd=" + crd + ", tx=" + tx + ']')); + + return; + } + } + + if (ctx.localNodeId().equals(crd.nodeId())) { + if (!initFut.isDone()) { + // Wait for the local coordinator init. + initFut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + requestSnapshotAsync(tx, lsnr); + } + }); + } + else if (tx != null) + lsnr.onResponse(assignTxSnapshot(0L)); + else + lsnr.onResponse(activeQueries.assignQueryCounter(ctx.localNodeId(), 0L)); + + return; + } + + // Send request to the remote coordinator. + UUID nodeId = crd.nodeId(); + + long id = futIdCntr.incrementAndGet(); + + Map map = snapLsnrs.get(nodeId), map0; + + if (map == null && (map0 = snapLsnrs.putIfAbsent(nodeId, map = new ConcurrentHashMap<>())) != null) + map = map0; + + map.put(id, lsnr); + + try { + sendMessage(nodeId, tx != null ? new MvccTxSnapshotRequest(id) : new MvccQuerySnapshotRequest(id)); + } + catch (IgniteCheckedException e) { + if (map.remove(id) != null) + lsnr.onError(e); + } + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture ackTxCommit(MvccSnapshot updateVer) { + return ackTxCommit(updateVer, null, 0L); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture ackTxCommit(MvccVersion updateVer, MvccSnapshot readSnapshot, + long qryId) { + assert updateVer != null; + + MvccCoordinator crd = curCrd; + + if (updateVer.coordinatorVersion() == crd.coordinatorVersion()) + return sendTxCommit(crd, createTxAckMessage(futIdCntr.incrementAndGet(), updateVer, readSnapshot, qryId)); + else if (readSnapshot != null) + ackQueryDone(readSnapshot, qryId); + + return new GridFinishedFuture<>(); + } + + /** {@inheritDoc} */ + @Override public void ackTxRollback(MvccVersion updateVer) { + assert updateVer != null; + + MvccCoordinator crd = curCrd; + + if (crd.coordinatorVersion() != updateVer.coordinatorVersion()) + return; + + MvccAckRequestTx msg = createTxAckMessage(-1, updateVer, null, 0L); + + msg.skipResponse(true); + + try { + sendMessage(crd.nodeId(), msg); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send tx rollback ack, node left [msg=" + msg + ", node=" + crd.nodeId() + ']'); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send tx rollback ack [msg=" + msg + ", node=" + crd.nodeId() + ']', e); + } + } + + /** {@inheritDoc} */ + @Override public void ackTxRollback(MvccVersion updateVer, MvccSnapshot readSnapshot, long qryTrackerId) { + assert updateVer != null; + + MvccCoordinator crd = curCrd; + + if (crd.coordinatorVersion() != updateVer.coordinatorVersion()) { + if (readSnapshot != null) + ackQueryDone(readSnapshot, qryTrackerId); + + return; + } + + MvccAckRequestTx msg = createTxAckMessage(-1, updateVer, readSnapshot, qryTrackerId); + + msg.skipResponse(true); + + try { + sendMessage(crd.nodeId(), msg); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send tx rollback ack, node left [msg=" + msg + ", node=" + crd.nodeId() + ']'); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send tx rollback ack [msg=" + msg + ", node=" + crd.nodeId() + ']', e); + } + } + + /** {@inheritDoc} */ + @Override public void ackQueryDone(MvccSnapshot snapshot, long qryId) { + assert snapshot != null; + + MvccCoordinator crd = currentCoordinator(); + + if (crd == null || crd.coordinatorVersion() == snapshot.coordinatorVersion() + && sendQueryDone(crd, new MvccAckRequestQueryCntr(queryTrackCounter(snapshot)))) + return; + + Message msg = new MvccAckRequestQueryId(qryId); + + do { + crd = currentCoordinator(); + } + while (!sendQueryDone(crd, msg)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture waitTxsFuture(UUID crdId, GridLongList txs) { + assert crdId != null; + assert txs != null && txs.size() > 0; + + WaitAckFuture fut = new WaitAckFuture(futIdCntr.incrementAndGet(), crdId, false); + + ackFuts.put(fut.id, fut); + + try { + sendMessage(crdId, new MvccWaitTxsRequest(fut.id, txs)); + } + catch (IgniteCheckedException e) { + if (ackFuts.remove(fut.id) != null) { + if (e instanceof ClusterTopologyCheckedException) + fut.onDone(); // No need to wait, new coordinator will be assigned, finish without error. + else + fut.onDone(e); + } + } + + return fut; + } + + /** {@inheritDoc} */ + // TODO: Proper use of diagnostic context. + @Override public void dumpDebugInfo(IgniteLogger log, @Nullable IgniteDiagnosticPrepareContext diagCtx) { + boolean first = true; + + for (Map map : snapLsnrs.values()) { + if (first) { + U.warn(log, "Pending mvcc listener: "); + + first = false; + } + + for (MvccSnapshotResponseListener lsnr : map.values()) { + U.warn(log, ">>> " + lsnr.toString()); + } + } + + first = true; + + for (WaitAckFuture waitAckFut : ackFuts.values()) { + if (first) { + U.warn(log, "Pending mvcc wait ack futures: "); + + first = false; + } + + U.warn(log, ">>> " + waitAckFut.toString()); + } + } + + /** + * Removes all less or equals to the given one records from Tx log. + * + * @param ver Version. + * @throws IgniteCheckedException If fails. + */ + void removeUntil(MvccVersion ver) throws IgniteCheckedException { + txLog.removeUntil(ver.coordinatorVersion(), ver.counter()); + } + + /** + * TODO IGNITE-7966 + * + * @return Data region configuration. + */ + private DataRegionConfiguration createTxLogRegion(DataStorageConfiguration dscfg) { + DataRegionConfiguration cfg = new DataRegionConfiguration(); + + cfg.setName(TX_LOG_CACHE_NAME); + cfg.setInitialSize(dscfg.getSystemRegionInitialSize()); + cfg.setMaxSize(dscfg.getSystemRegionMaxSize()); + cfg.setPersistenceEnabled(CU.isPersistenceEnabled(dscfg)); + return cfg; + } + + /** + * @return Data storage configuration. + */ + private DataStorageConfiguration dataStorageConfiguration() { + return ctx.config().getDataStorageConfiguration(); + } + + /** + * @param topVer Topology version. + * @return Coordinator version. + */ + private long coordinatorVersion(long topVer) { + return topVer + ctx.discovery().gridStartTime(); + } + + /** + * @return Active queries list. + */ + private GridLongList collectActiveQueryTrackers() { + assert curCrd != null; + + GridLongList activeQryTrackers = new GridLongList(); + + for (MvccQueryTracker tracker : activeTrackers.values()) { + long trackerId = tracker.onMvccCoordinatorChange(curCrd); + + if (trackerId != MVCC_TRACKER_ID_NA) + activeQryTrackers.add(trackerId); + } + + return activeQryTrackers; + } + + /** + * @return Counter. + */ + private MvccSnapshotResponse assignTxSnapshot(long futId) { + assert initFut.isDone(); + assert crdVer != 0; + assert ctx.localNodeId().equals(currentCoordinatorId()); + + MvccSnapshotResponse res = new MvccSnapshotResponse(); + + long ver, cleanup, tracking; + + synchronized (this) { + ver = mvccCntr.incrementAndGet(); + tracking = ver; + cleanup = committedCntr.get() + 1; + + for (Map.Entry txVer : activeTxs.entrySet()) { + cleanup = Math.min(txVer.getValue(), cleanup); + tracking = Math.min(txVer.getKey(), tracking); + + res.addTx(txVer.getKey()); + } + + boolean add = activeTxs.put(ver, tracking) == null; + + assert add : ver; + } + + long minQry = activeQueries.minimalQueryCounter(); + + if (minQry != -1) + cleanup = Math.min(cleanup, minQry); + + cleanup = prevCrdQueries.previousQueriesDone() ? cleanup - 1 : MVCC_COUNTER_NA; + + res.init(futId, crdVer, ver, MVCC_START_OP_CNTR, cleanup, tracking); + + return res; + } + + /** + * @param txCntr Counter assigned to transaction. + */ + private void onTxDone(Long txCntr, boolean committed) { + assert initFut.isDone(); + + GridFutureAdapter fut; + + synchronized (this) { + activeTxs.remove(txCntr); + + if (committed) + committedCntr.setIfGreater(txCntr); + } + + fut = waitTxFuts.remove(txCntr); + + if (fut != null) + fut.onDone(); + } + + /** + * @param mvccCntr Query counter. + */ + private void onQueryDone(UUID nodeId, Long mvccCntr) { + activeQueries.onQueryDone(nodeId, mvccCntr); + } + + /** + * @param futId Future ID. + * @param updateVer Update version. + * @param readSnapshot Optional read version. + * @param qryTrackerId Query tracker id. + * @return Message. + */ + private MvccAckRequestTx createTxAckMessage(long futId, MvccVersion updateVer, MvccSnapshot readSnapshot, + long qryTrackerId) { + if (readSnapshot == null) + return new MvccAckRequestTx(futId, updateVer.counter()); + else if (readSnapshot.coordinatorVersion() == updateVer.coordinatorVersion()) + return new MvccAckRequestTxAndQueryCntr(futId, updateVer.counter(), queryTrackCounter(readSnapshot)); + else + return new MvccAckRequestTxAndQueryId(futId, updateVer.counter(), qryTrackerId); + } + + /** + * @param mvccVer Read version. + * @return Tracker counter. + */ + private long queryTrackCounter(MvccSnapshot mvccVer) { + long trackCntr = mvccVer.counter(); + + MvccLongList txs = mvccVer.activeTransactions(); + + int size = txs.size(); + + for (int i = 0; i < size; i++) { + long txVer = txs.get(i); + + if (txVer < trackCntr) + trackCntr = txVer; + } + + return trackCntr; + } + + /** + * Launches vacuum workers and scheduler. + */ + void startVacuumWorkers() { + if (!ctx.clientNode()) { + synchronized (mux) { + if (vacuumWorkers == null) { + assert cleanupQueue == null; + + cleanupQueue = new LinkedBlockingQueue<>(); + + vacuumWorkers = new ArrayList<>(ctx.config().getMvccVacuumThreadCnt() + 1); + + vacuumWorkers.add(new VacuumScheduler(ctx, log, this)); + + for (int i = 0; i < ctx.config().getMvccVacuumThreadCnt(); i++) { + vacuumWorkers.add(new VacuumWorker(ctx, log, cleanupQueue)); + } + + for (GridWorker worker : vacuumWorkers) { + new IgniteThread(worker).start(); + } + + return; + } + } + + U.warn(log, "Attempting to start active vacuum."); + } + } + + /** + * Stops vacuum worker and scheduler. + */ + void stopVacuumWorkers() { + if (!ctx.clientNode()) { + List workers; + BlockingQueue queue; + + synchronized (mux) { + workers = vacuumWorkers; + queue = cleanupQueue; + + vacuumWorkers = null; + cleanupQueue = null; + } + + if (workers == null) { + U.warn(log, "Attempting to stop inactive vacuum."); + + return; + } + + assert queue != null; + + // Stop vacuum workers outside mutex to prevent deadlocks. + U.cancel(workers); + U.join(workers, log); + + if (!queue.isEmpty()) { + IgniteCheckedException ex = vacuumCancelledException(); + + for (VacuumTask task : queue) { + task.onDone(ex); + } + } + } + } + + /** + * Runs vacuum process. + * + * @return {@code Future} with {@link VacuumMetrics}. + */ + IgniteInternalFuture runVacuum() { + assert !ctx.clientNode(); + + MvccCoordinator crd0 = currentCoordinator(); + + if (Thread.currentThread().isInterrupted() || + crd0 == null || + crdVer == 0 && ctx.localNodeId().equals(crd0.nodeId())) + return new GridFinishedFuture<>(new VacuumMetrics()); + + final GridCompoundIdentityFuture res = + new GridCompoundIdentityFuture<>(new VacuumMetricsReducer()); + + MvccSnapshot snapshot; + + try { + // TODO IGNITE-8974 create special method for getting cleanup version only. + snapshot = tryRequestSnapshotLocal(DUMMY_TX); + } + catch (ClusterTopologyCheckedException e) { + throw new AssertionError(e); + } + + if (snapshot != null) + continueRunVacuum(res, snapshot); + else + requestSnapshotAsync(DUMMY_TX, new MvccSnapshotResponseListener() { + @Override public void onResponse(MvccSnapshot s) { + continueRunVacuum(res, s); + } + + @Override public void onError(IgniteCheckedException e) { + if (!(e instanceof ClusterTopologyCheckedException)) + completeWithException(res, e); + else { + if (log.isDebugEnabled()) + log.debug("Vacuum failed to receive an Mvcc snapshot. " + + "Need to retry on the stable topology. " + e.getMessage()); + + res.onDone(new VacuumMetrics()); + } + } + }); + + return res; + } + + /** + * For tests only. + * + * @return Vacuum error. + */ + Throwable vacuumError() { + return vacuumError; + } + + /** + * For tests only. + * + * @param e Vacuum error. + */ + void vacuumError(Throwable e) { + this.vacuumError = e; + } + + /** + * @param res Result. + * @param snapshot Snapshot. + */ + private void continueRunVacuum(GridCompoundIdentityFuture res, MvccSnapshot snapshot) { + ackTxCommit(snapshot) + .listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + Throwable err; + + if ((err = fut.error()) != null) { + U.error(log, "Vacuum error.", err); + + res.onDone(err); + } + else if (snapshot.cleanupVersion() <= MVCC_COUNTER_NA) + res.onDone(new VacuumMetrics()); + else { + try { + if (log.isDebugEnabled()) + log.debug("Started vacuum with cleanup version=" + snapshot.cleanupVersion() + '.'); + + synchronized (mux) { + if (cleanupQueue == null) { + res.onDone(vacuumCancelledException()); + + return; + } + + for (CacheGroupContext grp : ctx.cache().cacheGroups()) { + if (grp.mvccEnabled()) { + for (GridDhtLocalPartition part : grp.topology().localPartitions()) { + VacuumTask task = new VacuumTask(snapshot, part); + + cleanupQueue.offer(task); + + res.add(task); + } + } + } + } + + res.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + VacuumMetrics metrics = fut.get(); + + if (U.assertionsEnabled()) { + MvccCoordinator crd = currentCoordinator(); + + assert crd != null + && crd.coordinatorVersion() >= snapshot.coordinatorVersion(); + + for (TxKey key : waitMap.keySet()) { + assert key.major() == snapshot.coordinatorVersion() + && key.minor() > snapshot.cleanupVersion() + || key.major() > snapshot.coordinatorVersion(); + } + } + + txLog.removeUntil(snapshot.coordinatorVersion(), snapshot.cleanupVersion()); + + if (log.isDebugEnabled()) + log.debug("Vacuum completed. " + metrics); + } + catch (NodeStoppingException ignored) { + if (log.isDebugEnabled()) + log.debug("Cannot complete vacuum (node is stopping)."); + } + catch (Throwable e) { + U.error(log, "Vacuum error.", e); + } + } + }); + + res.markInitialized(); + } + catch (Throwable e) { + completeWithException(res, e); + } + } + } + }); + } + + /** */ + private void completeWithException(GridFutureAdapter fut, Throwable e) { + fut.onDone(e); + + if (e instanceof Error) + throw (Error)e; + } + + /** */ + @NotNull private IgniteCheckedException vacuumCancelledException() { + return new NodeStoppingException("Operation has been cancelled (node is stopping)."); + } + + /** + * @param nodeId Node ID. + * @param msg Message. + */ + private void sendFutureResponse(UUID nodeId, MvccWaitTxsRequest msg) { + try { + sendMessage(nodeId, new MvccFutureResponse(msg.futureId())); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send tx ack response, node left [msg=" + msg + ", node=" + nodeId + ']'); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send tx ack response [msg=" + msg + ", node=" + nodeId + ']', e); + } + } + + /** */ + @NotNull private IgniteInternalFuture sendTxCommit(MvccCoordinator crd, MvccAckRequestTx msg) { + WaitAckFuture fut = new WaitAckFuture(msg.futureId(), crd.nodeId(), true); + + ackFuts.put(fut.id, fut); + + try { + sendMessage(crd.nodeId(), msg); + } + catch (IgniteCheckedException e) { + if (ackFuts.remove(fut.id) != null) { + if (e instanceof ClusterTopologyCheckedException) { + if (log.isDebugEnabled()) + log.debug("Failed to send tx ack, node left [crd=" + crd + ", msg=" + msg + ']'); + + fut.onDone(); // No need to ack, finish without error. + } + else + + fut.onDone(e); + } + } + + return fut; + } + + /** + * @param crd Mvcc coordinator. + * @param msg Message. + * @return {@code True} if no need to resend the message to a new coordinator. + */ + private boolean sendQueryDone(MvccCoordinator crd, Message msg) { + if (crd == null) + return true; // no need to send ack; + + try { + sendMessage(crd.nodeId(), msg); + + return true; + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send query ack, node left [crd=" + crd + ", msg=" + msg + ']'); + + MvccCoordinator crd0 = currentCoordinator(); + + // Coordinator is unassigned or still the same. + return crd0 == null || crd.coordinatorVersion() == crd0.coordinatorVersion(); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send query ack [crd=" + crd + ", msg=" + msg + ']', e); + + return true; + } + } + + /** + * Send IO message. + * + * @param nodeId Node ID. + * @param msg Message. + */ + private void sendMessage(UUID nodeId, Message msg) throws IgniteCheckedException { + ctx.io().sendToGridTopic(nodeId, TOPIC_CACHE_COORDINATOR, msg, SYSTEM_POOL); + } + + /** + * @param nodeId Sender node ID. + * @param msg Message. + */ + private void processCoordinatorTxSnapshotRequest(UUID nodeId, MvccTxSnapshotRequest msg) { + ClusterNode node = ctx.discovery().node(nodeId); + + if (node == null) { + if (log.isDebugEnabled()) + log.debug("Ignore tx snapshot request processing, node left [msg=" + msg + ", node=" + nodeId + ']'); + + return; + } + + MvccSnapshotResponse res = assignTxSnapshot(msg.futureId()); + + try { + sendMessage(node.id(), res); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send tx snapshot response, node left [msg=" + msg + ", node=" + nodeId + ']'); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send tx snapshot response [msg=" + msg + ", node=" + nodeId + ']', e); + } + } + + /** + * @param nodeId Sender node ID. + * @param msg Message. + */ + private void processCoordinatorQuerySnapshotRequest(UUID nodeId, MvccQuerySnapshotRequest msg) { + ClusterNode node = ctx.discovery().node(nodeId); + + if (node == null) { + if (log.isDebugEnabled()) + log.debug("Ignore query counter request processing, node left [msg=" + msg + ", node=" + nodeId + ']'); + + return; + } + + MvccSnapshotResponse res = activeQueries.assignQueryCounter(nodeId, msg.futureId()); + + try { + sendMessage(node.id(), res); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send query counter response, node left [msg=" + msg + ", node=" + nodeId + ']'); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send query counter response [msg=" + msg + ", node=" + nodeId + ']', e); + + onQueryDone(nodeId, res.tracking()); + } + } + + /** + * @param nodeId Sender node ID. + * @param msg Message. + */ + private void processCoordinatorSnapshotResponse(UUID nodeId, MvccSnapshotResponse msg) { + Map map = snapLsnrs.get(nodeId); MvccSnapshotResponseListener lsnr; + + if (map != null && (lsnr = map.remove(msg.futureId())) != null) + lsnr.onResponse(msg); + else { + if (ctx.discovery().alive(nodeId)) + U.warn(log, "Failed to find query version future [node=" + nodeId + ", msg=" + msg + ']'); + else if (log.isDebugEnabled()) + log.debug("Failed to find query version future [node=" + nodeId + ", msg=" + msg + ']'); + } + } + + /** + * @param nodeId Node ID. + * @param msg Message. + */ + private void processCoordinatorQueryAckRequest(UUID nodeId, MvccAckRequestQueryCntr msg) { + onQueryDone(nodeId, msg.counter()); + } + + /** + * @param nodeId Node ID. + * @param msg Message. + */ + private void processNewCoordinatorQueryAckRequest(UUID nodeId, MvccAckRequestQueryId msg) { + prevCrdQueries.onQueryDone(nodeId, msg.queryTrackerId()); + } + + /** + * @param nodeId Sender node ID. + * @param msg Message. + */ + private void processCoordinatorTxAckRequest(UUID nodeId, MvccAckRequestTx msg) { + onTxDone(msg.txCounter(), msg.futureId() >= 0); + + if (msg.queryCounter() != MVCC_COUNTER_NA) + onQueryDone(nodeId, msg.queryCounter()); + else if (msg.queryTrackerId() != MVCC_TRACKER_ID_NA) + prevCrdQueries.onQueryDone(nodeId, msg.queryTrackerId()); + + if (!msg.skipResponse()) { + try { + sendMessage(nodeId, new MvccFutureResponse(msg.futureId())); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send tx ack response, node left [msg=" + msg + ", node=" + nodeId + ']'); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send tx ack response [msg=" + msg + ", node=" + nodeId + ']', e); + } + } + } + + /** + * @param nodeId Sender node ID. + * @param msg Message. + */ + private void processCoordinatorAckResponse(UUID nodeId, MvccFutureResponse msg) { + WaitAckFuture fut = ackFuts.remove(msg.futureId()); + + if (fut != null) + fut.onResponse(); + else { + if (ctx.discovery().alive(nodeId)) + U.warn(log, "Failed to find tx ack future [node=" + nodeId + ", msg=" + msg + ']'); + else if (log.isDebugEnabled()) + log.debug("Failed to find tx ack future [node=" + nodeId + ", msg=" + msg + ']'); + } + } + + /** + * @param nodeId Node ID. + * @param msg Message. + */ + @SuppressWarnings("unchecked") + private void processCoordinatorWaitTxsRequest(final UUID nodeId, final MvccWaitTxsRequest msg) { + GridLongList txs = msg.transactions(); + + GridCompoundFuture resFut = null; + + for (int i = 0; i < txs.size(); i++) { + Long txId = txs.get(i); + + GridFutureAdapter fut = waitTxFuts.get(txId); + + if (fut == null) { + GridFutureAdapter old = waitTxFuts.putIfAbsent(txId, fut = new GridFutureAdapter()); + + if (old != null) + fut = old; + } + + boolean isDone; + + synchronized (this) { + isDone = !activeTxs.containsKey(txId); + } + + if (isDone) + fut.onDone(); + + if (!fut.isDone()) { + if (resFut == null) + resFut = new GridCompoundFuture(); + + resFut.add(fut); + } + } + + if (resFut != null) + resFut.markInitialized(); + + if (resFut == null || resFut.isDone()) + sendFutureResponse(nodeId, msg); + else { + resFut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + sendFutureResponse(nodeId, msg); + } + }); + } + } + + /** + * @param nodeId Node ID. + * @param msg Message. + */ + private void processCoordinatorActiveQueriesMessage(UUID nodeId, MvccActiveQueriesMessage msg) { + prevCrdQueries.addNodeActiveQueries(nodeId, msg.activeQueries()); + } + + /** + * + */ + private class ActiveQueries { + /** */ + private final Map> activeQueries = new HashMap<>(); + + /** */ + private Long minQry; + + /** */ + private synchronized long minimalQueryCounter() { + return minQry == null ? -1 : minQry; + } + + /** */ + private synchronized MvccSnapshotResponse assignQueryCounter(UUID nodeId, long futId) { + MvccSnapshotResponse res = new MvccSnapshotResponse(); + + long ver, tracking; + + synchronized (MvccProcessorImpl.this) { + ver = committedCntr.get(); + tracking = ver; + + for (Long txVer : activeTxs.keySet()) { + if (txVer < ver) { + tracking = Math.min(txVer, tracking); + res.addTx(txVer); + } + } + } + + TreeMap nodeMap = activeQueries.get(nodeId); + + if (nodeMap == null) { + activeQueries.put(nodeId, nodeMap = new TreeMap<>()); + + nodeMap.put(tracking, new AtomicInteger(1)); + } + else { + AtomicInteger cntr = nodeMap.get(tracking); + + if (cntr == null) + nodeMap.put(tracking, new AtomicInteger(1)); + else + cntr.incrementAndGet(); + } + + if (minQry == null) + minQry = tracking; + + res.init(futId, crdVer, ver, MVCC_READ_OP_CNTR, MVCC_COUNTER_NA, tracking); + + return res; + } + + /** */ + private synchronized void onQueryDone(UUID nodeId, Long ver) { + TreeMap nodeMap = activeQueries.get(nodeId); + + if (nodeMap == null) + return; + + assert minQry != null; + + AtomicInteger cntr = nodeMap.get(ver); + + assert cntr != null && cntr.get() > 0 : "onQueryDone ver=" + ver; + + if (cntr.decrementAndGet() == 0) { + nodeMap.remove(ver); + + if (nodeMap.isEmpty()) + activeQueries.remove(nodeId); + + if (ver.equals(minQry)) + minQry = activeMinimal(); + } + } + + /** */ + private synchronized void onNodeFailed(UUID nodeId) { + activeQueries.remove(nodeId); + + minQry = activeMinimal(); + } + + /** */ + private Long activeMinimal() { + Long min = null; + + for (TreeMap s : activeQueries.values()) { + Long first = s.firstKey(); + + if (min == null || first < min) + min = first; + } + + return min; + } + } + + /** + * + */ + private class WaitAckFuture extends MvccFuture { + /** */ + private final long id; + + /** */ + final boolean ackTx; + + /** + * @param id Future ID. + * @param nodeId Coordinator node ID. + * @param ackTx {@code True} if ack tx commit, {@code false} if waits for previous txs. + */ + WaitAckFuture(long id, UUID nodeId, boolean ackTx) { + super(nodeId); + + this.id = id; + this.ackTx = ackTx; + } + + /** + * + */ + void onResponse() { + onDone(); + } + + /** + * @param nodeId Failed node ID. + */ + void onNodeLeft(UUID nodeId) { + if (crdId.equals(nodeId) && ackFuts.remove(id) != null) + onDone(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(WaitAckFuture.class, this, super.toString()); + } + } + + /** + * + */ + private class CacheCoordinatorNodeFailListener implements GridLocalEventListener { + /** {@inheritDoc} */ + @Override public void onEvent(Event evt) { + assert evt instanceof DiscoveryEvent : evt; + + DiscoveryEvent discoEvt = (DiscoveryEvent)evt; + + UUID nodeId = discoEvt.eventNode().id(); + + Map map = snapLsnrs.remove(nodeId); + + if (map != null) { + ClusterTopologyCheckedException ex = new ClusterTopologyCheckedException("Failed to request mvcc " + + "version, coordinator failed: " + nodeId); + + MvccSnapshotResponseListener lsnr; + + for (Long id : map.keySet()) { + if ((lsnr = map.remove(id)) != null) + lsnr.onError(ex); + } + } + + for (WaitAckFuture fut : ackFuts.values()) + fut.onNodeLeft(nodeId); + + activeQueries.onNodeFailed(nodeId); + + prevCrdQueries.onNodeFailed(nodeId); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "CacheCoordinatorDiscoveryListener[]"; + } + } + + /** + * + */ + private class CoordinatorMessageListener implements GridMessageListener { + /** {@inheritDoc} */ + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { + MvccMessage msg0 = (MvccMessage)msg; + + if (msg0.waitForCoordinatorInit() && !initFut.isDone()) { + initFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture future) { + assert crdVer != 0L; + + processMessage(nodeId, msg); + } + }); + } + else + processMessage(nodeId, msg); + } + + /** + * Processes mvcc message. + * + * @param nodeId Node id. + * @param msg Message. + */ + private void processMessage(UUID nodeId, Object msg) { + if (msg instanceof MvccTxSnapshotRequest) + processCoordinatorTxSnapshotRequest(nodeId, (MvccTxSnapshotRequest)msg); + else if (msg instanceof MvccAckRequestTx) + processCoordinatorTxAckRequest(nodeId, (MvccAckRequestTx)msg); + else if (msg instanceof MvccFutureResponse) + processCoordinatorAckResponse(nodeId, (MvccFutureResponse)msg); + else if (msg instanceof MvccAckRequestQueryCntr) + processCoordinatorQueryAckRequest(nodeId, (MvccAckRequestQueryCntr)msg); + else if (msg instanceof MvccQuerySnapshotRequest) + processCoordinatorQuerySnapshotRequest(nodeId, (MvccQuerySnapshotRequest)msg); + else if (msg instanceof MvccSnapshotResponse) + processCoordinatorSnapshotResponse(nodeId, (MvccSnapshotResponse)msg); + else if (msg instanceof MvccWaitTxsRequest) + processCoordinatorWaitTxsRequest(nodeId, (MvccWaitTxsRequest)msg); + else if (msg instanceof MvccAckRequestQueryId) + processNewCoordinatorQueryAckRequest(nodeId, (MvccAckRequestQueryId)msg); + else if (msg instanceof MvccActiveQueriesMessage) + processCoordinatorActiveQueriesMessage(nodeId, (MvccActiveQueriesMessage)msg); + else + U.warn(log, "Unexpected message received [node=" + nodeId + ", msg=" + msg + ']'); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "CoordinatorMessageListener[]"; + } + } + + /** */ + private interface Waiter { + /** + * @param ctx Grid kernal context. + */ + void run(GridKernalContext ctx); + + /** + * @param other Another waiter. + * @return New compound waiter. + */ + Waiter concat(Waiter other); + + /** + * @return {@code True} if there is an active local transaction + */ + boolean hasLocalTransaction(); + + /** + * @return {@code True} if it is a compound waiter. + */ + boolean compound(); + } + + /** */ + private static class LockFuture extends GridFutureAdapter implements Waiter, Runnable { + /** */ + private final byte plc; + + /** + * @param plc Pool policy. + */ + LockFuture(byte plc) { + this.plc = plc; + } + + /** {@inheritDoc} */ + @Override public void run() { + onDone(); + } + + /** {@inheritDoc} */ + @Override public void run(GridKernalContext ctx) { + try { + ctx.pools().poolForPolicy(plc).execute(this); + } + catch (IgniteCheckedException e) { + U.error(ctx.log(LockFuture.class), e); + } + } + + /** {@inheritDoc} */ + @Override public Waiter concat(Waiter other) { + return new CompoundWaiterNoLocal(this, other); + } + + /** {@inheritDoc} */ + @Override public boolean hasLocalTransaction() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean compound() { + return false; + } + } + + /** */ + private static class LocalTransactionMarker implements Waiter { + /** {@inheritDoc} */ + @Override public void run(GridKernalContext ctx) { + // No-op + } + + /** {@inheritDoc} */ + @Override public Waiter concat(Waiter other) { + return new CompoundWaiter(other); + } + + /** {@inheritDoc} */ + @Override public boolean hasLocalTransaction() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean compound() { + return false; + } + } + + /** */ + @SuppressWarnings("unchecked") + private static class CompoundWaiter implements Waiter { + /** */ + private final Object inner; + + /** + * @param waiter Waiter to wrap. + */ + private CompoundWaiter(Waiter waiter) { + inner = waiter.compound() ? ((CompoundWaiter)waiter).inner : waiter; + } + + /** + * @param first First waiter. + * @param second Second waiter. + */ + private CompoundWaiter(Waiter first, Waiter second) { + ArrayList list = new ArrayList<>(); + + add(list, first); + add(list, second); + + inner = list; + } + + /** */ + private void add(List to, Waiter waiter) { + if (!waiter.compound()) + to.add(waiter); + else if (((CompoundWaiter)waiter).inner.getClass() == ArrayList.class) + to.addAll((List)((CompoundWaiter)waiter).inner); + else + to.add((Waiter)((CompoundWaiter)waiter).inner); + } + + /** {@inheritDoc} */ + @Override public void run(GridKernalContext ctx) { + if (inner.getClass() == ArrayList.class) { + for (Waiter waiter : (List)inner) { + waiter.run(ctx); + } + } + else + ((Waiter)inner).run(ctx); + } + + /** {@inheritDoc} */ + @Override public Waiter concat(Waiter other) { + return new CompoundWaiter(this, other); + } + + /** {@inheritDoc} */ + @Override public boolean hasLocalTransaction() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean compound() { + return true; + } + } + + /** */ + private static class CompoundWaiterNoLocal extends CompoundWaiter { + /** + * @param first First waiter. + * @param second Second waiter. + */ + private CompoundWaiterNoLocal(Waiter first, Waiter second) { + super(first, second); + } + + /** {@inheritDoc} */ + @Override public Waiter concat(Waiter other) { + return new CompoundWaiterNoLocal(this, other); + } + + /** {@inheritDoc} */ + @Override public boolean hasLocalTransaction() { + return false; + } + } + + /** + * Mvcc garbage collection scheduler. + */ + private static class VacuumScheduler extends GridWorker { + /** */ + private final static long VACUUM_TIMEOUT = 60_000; + + /** */ + private final long interval; + + /** */ + private final MvccProcessorImpl prc; + + /** + * @param ctx Kernal context. + * @param log Logger. + * @param prc Mvcc processor. + */ + VacuumScheduler(GridKernalContext ctx, IgniteLogger log, MvccProcessorImpl prc) { + super(ctx.igniteInstanceName(), "vacuum-scheduler", log); + + this.interval = ctx.config().getMvccVacuumTimeInterval(); + this.prc = prc; + } + + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + U.sleep(interval); // initial delay + + while (!isCancelled()) { + long nextScheduledTime = U.currentTimeMillis() + interval; + + try { + IgniteInternalFuture fut = prc.runVacuum(); + + if (log.isDebugEnabled()) + log.debug("Vacuum started by scheduler."); + + while (true) { + try { + fut.get(VACUUM_TIMEOUT); + + break; + } + catch (IgniteFutureTimeoutCheckedException e) { + U.warn(log, "Failed to wait for vacuum complete. Consider increasing vacuum workers count."); + } + } + } + catch (IgniteInterruptedCheckedException e) { + throw e; + } + catch (Throwable e) { + prc.vacuumError(e); + + if (e instanceof Error) + throw (Error) e; + } + + long delay = nextScheduledTime - U.currentTimeMillis(); + + if (delay > 0) + U.sleep(delay); + } + } + } + + /** + * Vacuum worker. + */ + private static class VacuumWorker extends GridWorker { + /** */ + private final BlockingQueue cleanupQueue; + + /** + * @param ctx Kernal context. + * @param log Logger. + * @param cleanupQueue Cleanup tasks queue. + */ + VacuumWorker(GridKernalContext ctx, IgniteLogger log, BlockingQueue cleanupQueue) { + super(ctx.igniteInstanceName(), "vacuum-cleaner", log); + + this.cleanupQueue = cleanupQueue; + } + + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + while (!isCancelled()) { + VacuumTask task = cleanupQueue.take(); + + try { + if (task.part().state() != OWNING) { + task.part().group().preloader().rebalanceFuture() + .listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture future) { + cleanupQueue.add(task); + } + }); + + continue; + } + + task.onDone(processPartition(task)); + } + catch (IgniteInterruptedCheckedException e) { + throw e; // Cancelled. + } + catch (Throwable e) { + task.onDone(e); + + if (e instanceof Error) + throw (Error) e; + } + } + } + + /** + * Process partition. + * + * @param task VacuumTask. + * @throws IgniteCheckedException If failed. + */ + private VacuumMetrics processPartition(VacuumTask task) throws IgniteCheckedException { + long startNanoTime = System.nanoTime(); + + GridDhtLocalPartition part = task.part(); + + VacuumMetrics metrics = new VacuumMetrics(); + + if (part == null || part.state() != OWNING || !part.reserve()) + return metrics; + + try { + GridCursor cursor = part.dataStore().cursor(KEY_ONLY); + + KeyCacheObject prevKey = null; Object rest = null; + + List cleanupRows = null; + + MvccSnapshot snapshot = task.snapshot(); + + GridCacheContext cctx = null; int curCacheId = CU.UNDEFINED_CACHE_ID; + + boolean shared = part.group().sharedGroup(); + + if (!shared) + cctx = part.group().singleCacheContext(); + + while (cursor.next()) { + if (isCancelled()) + throw new IgniteInterruptedCheckedException("Operation has been cancelled."); + + MvccDataRow row = (MvccDataRow)cursor.get(); + + if (prevKey == null) + prevKey = row.key(); + + if (cctx == null) { + assert shared; + + curCacheId = row.cacheId(); + cctx = part.group().shared().cacheContext(curCacheId); + } + + if (!prevKey.equals(row.key()) || (shared && curCacheId != row.cacheId())) { + if (rest != null || !F.isEmpty(cleanupRows)) + cleanup(part, prevKey, cleanupRows, rest, cctx, metrics); + + cleanupRows = null; rest = null; + + if (shared && curCacheId != row.cacheId()) + cctx = part.group().shared().cacheContext(curCacheId = row.cacheId()); + + prevKey = row.key(); + } + + if (canClean(row, snapshot, cctx)) + cleanupRows = addRow(cleanupRows, row); + else if (actualize(cctx, row, snapshot)) + rest = addRest(rest, row); + + metrics.addScannedRowsCount(1); + } + + if (rest != null || !F.isEmpty(cleanupRows)) + cleanup(part, prevKey, cleanupRows, rest, cctx, metrics); + + metrics.addSearchNanoTime(System.nanoTime() - startNanoTime - metrics.cleanupNanoTime()); + + return metrics; + } + finally { + part.release(); + } + } + + /** */ + @SuppressWarnings("unchecked") + @NotNull private Object addRest(@Nullable Object rest, MvccDataRow row) { + if (rest == null) + rest = row; + else if (rest.getClass() == ArrayList.class) + ((List)rest).add(row); + else { + ArrayList list = new ArrayList(); + + list.add(rest); + list.add(row); + + rest = list; + } + + return rest; + } + + /** + * @param rows Collection of rows. + * @param row Row to add. + * @return Collection of rows. + */ + @NotNull private List addRow(@Nullable List rows, MvccDataRow row) { + if (rows == null) + rows = new ArrayList<>(); + + rows.add(new MvccLinkAwareSearchRow(row.cacheId(), row.key(), row.mvccCoordinatorVersion(), + row.mvccCounter(), row.mvccOperationCounter(), row.link())); + + return rows; + } + + /** + * @param row Mvcc row to check. + * @param snapshot Cleanup version to compare with. + * @param cctx Cache context. + * @throws IgniteCheckedException If failed. + */ + private boolean canClean(MvccDataRow row, MvccSnapshot snapshot, + GridCacheContext cctx) throws IgniteCheckedException { + // Row can be safely cleaned if it has ABORTED min version or COMMITTED and less than cleanup one max version. + return compare(row, snapshot.coordinatorVersion(), snapshot.cleanupVersion()) <= 0 + && hasNewVersion(row) && MvccUtils.compareNewVersion(row, snapshot.coordinatorVersion(), snapshot.cleanupVersion()) <= 0 + && MvccUtils.state(cctx, row.newMvccCoordinatorVersion(), row.newMvccCounter(), + row.newMvccOperationCounter() | (row.newMvccTxState() << PageIO.MVCC_HINTS_BIT_OFF)) == TxState.COMMITTED + || MvccUtils.state(cctx, row.mvccCoordinatorVersion(), row.mvccCounter(), + row.mvccOperationCounter() | (row.mvccTxState() << PageIO.MVCC_HINTS_BIT_OFF)) == TxState.ABORTED; + } + + /** */ + private boolean actualize(GridCacheContext cctx, MvccDataRow row, + MvccSnapshot snapshot) throws IgniteCheckedException { + return isVisible(cctx, snapshot, row.mvccCoordinatorVersion(), row.mvccCounter(), row.mvccOperationCounter(), false) + && (row.mvccTxState() == TxState.NA || (row.newMvccCoordinatorVersion() != MVCC_CRD_COUNTER_NA && row.newMvccTxState() == TxState.NA)); + } + + /** + * + * @param part Local partition. + * @param key Key. + * @param cleanupRows Cleanup rows. + * @param cctx Cache context. + * @param metrics Vacuum metrics. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("unchecked") + private void cleanup(GridDhtLocalPartition part, KeyCacheObject key, List cleanupRows, + Object rest, GridCacheContext cctx, VacuumMetrics metrics) throws IgniteCheckedException { + assert key != null && cctx != null && (!F.isEmpty(cleanupRows) || rest != null); + + long cleanupStartNanoTime = System.nanoTime(); + + GridCacheEntryEx entry = cctx.cache().entryEx(key); + + while (true) { + entry.lockEntry(); + + if (!entry.obsolete()) + break; + + entry.unlockEntry(); + + entry = cctx.cache().entryEx(key); + } + + cctx.shared().database().checkpointReadLock(); + + int cleaned = 0; + + try { + if (cleanupRows != null) + cleaned = part.dataStore().cleanup(cctx, cleanupRows); + + if (rest != null) { + if (rest.getClass() == ArrayList.class) { + for (MvccDataRow row : ((List)rest)) { + part.dataStore().updateTxState(cctx, row); + } + } + else + part.dataStore().updateTxState(cctx, (MvccDataRow)rest); + } + } + finally { + cctx.shared().database().checkpointReadUnlock(); + + entry.unlockEntry(); + cctx.evicts().touch(entry, AffinityTopologyVersion.NONE); + + metrics.addCleanupNanoTime(System.nanoTime() - cleanupStartNanoTime); + metrics.addCleanupRowsCnt(cleaned); + } + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTracker.java new file mode 100644 index 0000000000000..f143a43492c28 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTracker.java @@ -0,0 +1,102 @@ +/* + * 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.mvcc; + +import java.util.concurrent.atomic.AtomicLong; +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.distributed.near.GridNearTxLocal; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Mvcc tracker. + */ +public interface MvccQueryTracker { + /** */ + public static final AtomicLong ID_CNTR = new AtomicLong(); + + /** */ + public static final long MVCC_TRACKER_ID_NA = -1; + + /** + * @return Tracker id. + */ + public long id(); + + /** + * @return Requested MVCC snapshot. + */ + public MvccSnapshot snapshot(); + + /** + * @return Cache context. + */ + public GridCacheContext context(); + + /** + * @return Topology version. + */ + public AffinityTopologyVersion topologyVersion(); + + /** + * Requests version on coordinator. + * + * @return Future to wait for result. + */ + public IgniteInternalFuture requestSnapshot(); + + /** + * Requests version on coordinator. + * + * @param topVer Topology version. + * @return Future to wait for result. + */ + public IgniteInternalFuture requestSnapshot(@NotNull AffinityTopologyVersion topVer); + + /** + * Requests version on coordinator. + * + * @param topVer Topology version. + * @param lsnr Response listener. + */ + public void requestSnapshot(@NotNull AffinityTopologyVersion topVer, @NotNull MvccSnapshotResponseListener lsnr); + + /** + * Marks tracker as done. + */ + public void onDone(); + + /** + * Marks tracker as done. + * + * @param tx Transaction. + * @param commit Commit flag. + * @return Acknowledge future. + */ + @Nullable public IgniteInternalFuture onDone(@NotNull GridNearTxLocal tx, boolean commit); + + /** + * Mvcc coordinator change callback. + * + * @param newCrd New mvcc coordinator. + * @return Query id if exists. + */ + long onMvccCoordinatorChange(MvccCoordinator newCrd); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java new file mode 100644 index 0000000000000..f46d1e0de69f4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccQueryTrackerImpl.java @@ -0,0 +1,348 @@ +/* + * 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.mvcc; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.lang.IgniteInClosure; +import org.jetbrains.annotations.NotNull; + +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.noCoordinatorError; + +/** + * Tracker used for an optimistic tx and not-in-tx queries. + */ +@SuppressWarnings("unchecked") +public class MvccQueryTrackerImpl implements MvccQueryTracker { + /** */ + @GridToStringExclude + private final GridCacheContext cctx; + + /** */ + @GridToStringExclude + private final IgniteLogger log; + + /** */ + @GridToStringExclude + private long crdVer; + + /** */ + private final long id; + + /** */ + private MvccSnapshot snapshot; + + /** */ + private volatile AffinityTopologyVersion topVer; + + /** */ + private final boolean canRemap; + + /** + * @param cctx Cache context. + */ + public MvccQueryTrackerImpl(GridCacheContext cctx) { + this(cctx, true); + } + + /** + * @param cctx Cache context. + * @param canRemap {@code True} if tracker can remap on coordinator fail. + */ + public MvccQueryTrackerImpl(GridCacheContext cctx, boolean canRemap) { + this.cctx = cctx; + this.id = ID_CNTR.incrementAndGet(); + this.canRemap = canRemap; + + log = cctx.logger(getClass()); + } + + /** {@inheritDoc} */ + @Override public long id() { + return id; + } + + /** {@inheritDoc} */ + @Override public synchronized MvccSnapshot snapshot() { + return snapshot; + } + + /** {@inheritDoc} */ + @Override public GridCacheContext context() { + return cctx; + } + + /** {@inheritDoc} */ + @Override public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshot() { + MvccSnapshot snapshot; MvccSnapshotFuture fut; + + if ((snapshot = snapshot()) != null) + return new GridFinishedFuture<>(snapshot); + + requestSnapshot0(cctx.shared().exchange().readyAffinityVersion(), fut = new MvccSnapshotFuture()); + + return fut; + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshot(@NotNull AffinityTopologyVersion topVer) { + MvccSnapshot snapshot; MvccSnapshotFuture fut; + + if ((snapshot = snapshot()) != null) + return new GridFinishedFuture<>(snapshot); + + requestSnapshot0(topVer, fut = new MvccSnapshotFuture()); + + return fut; + } + + /** {@inheritDoc} */ + @Override public void requestSnapshot(@NotNull AffinityTopologyVersion topVer, @NotNull MvccSnapshotResponseListener lsnr) { + MvccSnapshot snapshot = snapshot(); + + if (snapshot != null) + lsnr.onResponse(snapshot); + else + requestSnapshot0(topVer, lsnr); + } + + /** {@inheritDoc} */ + @Override public void onDone() { + MvccProcessor prc = cctx.shared().coordinators(); + + MvccSnapshot snapshot = snapshot(); + + if (snapshot != null) { + prc.removeQueryTracker(id); + + prc.ackQueryDone(snapshot, id); + } + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture onDone(@NotNull GridNearTxLocal tx, boolean commit) { + MvccSnapshot snapshot = snapshot(), txSnapshot = tx.mvccSnapshot(); + + if (snapshot == null && txSnapshot == null) + return commit ? new GridFinishedFuture<>() : null; + + MvccProcessor prc = cctx.shared().coordinators(); + + if (snapshot != null) + prc.removeQueryTracker(id); + + if (txSnapshot == null) + prc.ackQueryDone(snapshot, id); + else if (commit) + return prc.ackTxCommit(txSnapshot, snapshot, id); + else + prc.ackTxRollback(txSnapshot, snapshot, id); + + return null; + } + + /** {@inheritDoc} */ + @Override public synchronized long onMvccCoordinatorChange(MvccCoordinator newCrd) { + if (snapshot != null) { + assert crdVer != 0 : this; + + if (crdVer != newCrd.coordinatorVersion()) { + crdVer = newCrd.coordinatorVersion(); + + return id; + } + else + return MVCC_TRACKER_ID_NA; + } + else if (crdVer != 0) + crdVer = 0; // Mark for remap. + + return MVCC_TRACKER_ID_NA; + } + + /** */ + private void requestSnapshot0(AffinityTopologyVersion topVer, MvccSnapshotResponseListener lsnr) { + if (checkTopology(topVer, lsnr = decorate(lsnr))) { + try { + MvccSnapshot snapshot = cctx.shared().coordinators().tryRequestSnapshotLocal(); + + if (snapshot == null) + cctx.shared().coordinators().requestSnapshotAsync(lsnr); + else + lsnr.onResponse(snapshot); + } + catch (ClusterTopologyCheckedException e) { + lsnr.onError(e); + } + } + } + + /** */ + private MvccSnapshotResponseListener decorate(MvccSnapshotResponseListener lsnr) { + assert lsnr != null; + + if (lsnr.getClass() == ListenerDecorator.class) + return lsnr; + + return new ListenerDecorator(lsnr); + } + + /** + * Validates if mvcc snapshot could be requested on the given topology. + * + * @return {@code True} if topology is valid. + */ + private boolean checkTopology(AffinityTopologyVersion topVer, MvccSnapshotResponseListener lsnr) { + MvccCoordinator crd = cctx.affinity().mvccCoordinator(topVer); + + if (crd == null) { + lsnr.onError(noCoordinatorError(topVer)); + + return false; + } + + this.topVer = topVer; + + synchronized (this) { + crdVer = crd.coordinatorVersion(); + } + + MvccCoordinator curCrd = cctx.topology().mvccCoordinator(); + + if (!crd.equals(curCrd)) { + assert cctx.topology().topologyVersionFuture().initialVersion().compareTo(topVer) > 0; + + tryRemap(lsnr); + + return false; + } + + return true; + } + + /** */ + private void tryRemap(MvccSnapshotResponseListener lsnr) { + if (!canRemap) { + lsnr.onError(new ClusterTopologyCheckedException("Failed to request mvcc version, coordinator failed.")); + + return; + } + + IgniteInternalFuture waitFut = + cctx.shared().exchange().affinityReadyFuture(topVer.nextMinorVersion()); + + if (waitFut == null) + requestSnapshot(cctx.shared().exchange().readyAffinityVersion(), lsnr); + else { + waitFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + requestSnapshot(fut.get(), lsnr); + } + catch (IgniteCheckedException e) { + lsnr.onError(e); + } + } + }); + } + } + + /** + * @param res Response. + * @param lsnr Response listener. + * @return {@code false} if need to remap. + */ + private boolean onResponse0(@NotNull MvccSnapshot res, MvccSnapshotResponseListener lsnr) { + boolean needRemap = false; + + synchronized (this) { + assert snapshot() == null : "[this=" + this + ", rcvdVer=" + res + "]"; + + if (crdVer != 0) { + this.snapshot = res; + } + else + needRemap = true; + } + + if (needRemap) { // Coordinator failed or reassigned, need remap. + tryRemap(lsnr); + + return false; + } + + cctx.shared().coordinators().addQueryTracker(this); + + return true; + } + + /** + * @param e Exception. + * @param lsnr Response listener. + * @return {@code false} if need to remap. + */ + private boolean onError0(IgniteCheckedException e, MvccSnapshotResponseListener lsnr) { + if (e instanceof ClusterTopologyCheckedException && canRemap) { + if (e instanceof ClusterTopologyServerNotFoundException) + return true; // No Mvcc coordinator assigned + + if (log.isDebugEnabled()) + log.debug("Mvcc coordinator failed, need remap: " + e); + + tryRemap(lsnr); + + return false; + } + + return true; + } + + /** */ + private final class ListenerDecorator implements MvccSnapshotResponseListener { + /** */ + private final MvccSnapshotResponseListener lsnr; + + /** */ + private ListenerDecorator(MvccSnapshotResponseListener lsnr) { + this.lsnr = lsnr; + } + + @Override public void onResponse(MvccSnapshot res) { + if (onResponse0(res, this)) + lsnr.onResponse(res); + } + + @Override public void onError(IgniteCheckedException e) { + if (onError0(e, this)) + lsnr.onError(e); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshot.java new file mode 100644 index 0000000000000..5ed743a50983b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshot.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.plugin.extensions.communication.Message; + +/** + * MVCC snapshot which holds the following information: + * - Current MVCC version which should be used for visibility checks + * - List of active transactions which should not be visible to current transaction + * - Cleanup version which is used to help vacuum process. + */ +public interface MvccSnapshot extends MvccVersion, Message { + /** + * @return Active transactions. + */ + public MvccLongList activeTransactions(); + + /** + * @return Cleanup version (all smaller versions are safe to remove). + */ + public long cleanupVersion(); + + /** + * @return Version without active transactions. + */ + public MvccSnapshot withoutActiveTransactions(); + + /** + * Increments operation counter. + */ + public void incrementOperationCounter(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotFuture.java new file mode 100644 index 0000000000000..934ff2fc8b163 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotFuture.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +public class MvccSnapshotFuture extends MvccFuture implements MvccSnapshotResponseListener { + /** {@inheritDoc} */ + @Override public void onResponse(MvccSnapshot res) { + assert res != null; + + onDone(res); + } + + /** {@inheritDoc} */ + @Override public void onError(IgniteCheckedException err) { + onDone(err); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccSnapshotFuture.class, this, super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotResponseListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotResponseListener.java new file mode 100644 index 0000000000000..e0bf448fc3f74 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotResponseListener.java @@ -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. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.IgniteCheckedException; + +/** + * + */ +public interface MvccSnapshotResponseListener { + /** + * @param res Version. + */ + public void onResponse(MvccSnapshot res); + + /** + * @param e Error. + */ + public void onError(IgniteCheckedException e); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotWithoutTxs.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotWithoutTxs.java new file mode 100644 index 0000000000000..5be631739fe8c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccSnapshotWithoutTxs.java @@ -0,0 +1,204 @@ +/* + * 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.mvcc; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * + */ +public class MvccSnapshotWithoutTxs implements MvccSnapshot { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long crdVer; + + /** */ + private long cntr; + + /** */ + private long cleanupVer; + + /** */ + private int opCntr; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccSnapshotWithoutTxs() { + // No-op. + } + + /** + * @param crdVer Coordinator version. + * @param cntr Counter. + * @param cleanupVer Cleanup version. + */ + public MvccSnapshotWithoutTxs(long crdVer, long cntr, int opCntr, long cleanupVer) { + this.crdVer = crdVer; + this.cntr = cntr; + this.cleanupVer = cleanupVer; + this.opCntr = opCntr; + } + + /** {@inheritDoc} */ + @Override public MvccLongList activeTransactions() { + return MvccEmptyLongList.INSTANCE; + } + + /** {@inheritDoc} */ + @Override public long coordinatorVersion() { + return crdVer; + } + + /** {@inheritDoc} */ + @Override public long cleanupVersion() { + return cleanupVer; + } + + /** {@inheritDoc} */ + @Override public long counter() { + return cntr; + } + + /** {@inheritDoc} */ + @Override public int operationCounter() { + return opCntr; + } + + /** {@inheritDoc} */ + @Override public void incrementOperationCounter() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot withoutActiveTransactions() { + return this; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("cleanupVer", cleanupVer)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeLong("cntr", cntr)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeLong("crdVer", crdVer)) + return false; + + writer.incrementState(); + + case 3: + if (!writer.writeInt("opCntr", opCntr)) + 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: + cleanupVer = reader.readLong("cleanupVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + cntr = reader.readLong("cntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + crdVer = reader.readLong("crdVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 3: + opCntr = reader.readInt("opCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccSnapshotWithoutTxs.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 150; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 4; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccSnapshotWithoutTxs.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java new file mode 100644 index 0000000000000..17804c49887a7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.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.processors.cache.mvcc; + +/** + * Interface for objects aware theirs mvcc update version. + */ +public interface MvccUpdateVersionAware { + /** + * @return New mvcc coordinator version. + */ + public long newMvccCoordinatorVersion(); + + /** + * @return New mvcc counter. + */ + public long newMvccCounter(); + + /** + * @return New mvcc operation counter. + */ + public int newMvccOperationCounter(); + + /** + * @return New Tx state. + */ + public byte newMvccTxState(); + + /** + * Copies new MVCC version + * @param other Object to copy version from. + */ + public default void newMvccVersion(MvccUpdateVersionAware other) { + newMvccVersion(other.newMvccCoordinatorVersion(), other.newMvccCounter(), other.newMvccOperationCounter()); + } + + /** + * Sets new MVCC version + * @param ver MVCC version. + */ + public default void newMvccVersion(MvccVersion ver) { + newMvccVersion(ver.coordinatorVersion(), ver.counter(), ver.operationCounter()); + } + + /** + * Sets new mvcc version. + * @param crd New mvcc coordinator version. + * @param cntr New mvcc counter. + * @param opCntr New mvcc operation counter. + */ + public default void newMvccVersion(long crd, long cntr, int opCntr) { + throw new UnsupportedOperationException(); + } + + /** + * @return New mvcc version. + */ + public default MvccVersion newMvccVersion() { + return new MvccVersionImpl(newMvccCoordinatorVersion(), newMvccCounter(), newMvccOperationCounter()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java new file mode 100644 index 0000000000000..33f457db5f231 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java @@ -0,0 +1,882 @@ +/* + * 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.mvcc; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.transactions.IgniteTxMvccVersionCheckedException; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.transactions.TransactionState; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId; +import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_BIT_OFF; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_MASK; +import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.TRANSACTION_COMPLETED; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Utils for MVCC. + */ +public class MvccUtils { + /** */ + public static final long MVCC_CRD_COUNTER_NA = 0L; + /** */ + public static final long MVCC_CRD_START_CNTR = 1L; + /** */ + public static final long MVCC_COUNTER_NA = 0L; + /** */ + public static final long MVCC_INITIAL_CNTR = 1L; + /** */ + public static final long MVCC_START_CNTR = 3L; + /** */ + public static final int MVCC_OP_COUNTER_NA = 0; + /** */ + public static final int MVCC_START_OP_CNTR = 1; + /** */ + public static final int MVCC_READ_OP_CNTR = ~MVCC_HINTS_MASK; + + /** */ + public static final int MVCC_INVISIBLE = 0; + /** */ + public static final int MVCC_VISIBLE_REMOVED = 1; + /** */ + public static final int MVCC_VISIBLE = 2; + + /** */ + public static final MvccVersion INITIAL_VERSION = + mvccVersion(MVCC_CRD_START_CNTR, MVCC_INITIAL_CNTR, MVCC_START_OP_CNTR); + + /** */ + public static final MvccVersion MVCC_VERSION_NA = + mvccVersion(MVCC_CRD_COUNTER_NA, MVCC_COUNTER_NA, MVCC_OP_COUNTER_NA); + + /** */ + private static final MvccClosure getVisibleState = new GetVisibleState(); + + /** */ + private static final MvccClosure isVisible = new IsVisible(); + + /** */ + private static final MvccClosure getNewVer = new GetNewVersion(); + + /** + * + */ + private MvccUtils(){ + } + + /** + * @param ctx Kernal context. + * @return Newly created Mvcc processor. + */ + public static MvccProcessor createProcessor(GridKernalContext ctx) { + return mvccEnabled(ctx) ? new MvccProcessorImpl(ctx) : new NoOpMvccProcessor(ctx); + } + + /** + * @param cctx Cache context. + * @param mvccCrd Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @param snapshot Snapshot. + * @return {@code True} if transaction is active. + * @see TxState + * @throws IgniteCheckedException If failed. + */ + public static boolean isActive(GridCacheContext cctx, long mvccCrd, long mvccCntr, MvccSnapshot snapshot) + throws IgniteCheckedException { + if (isVisible(cctx, snapshot, mvccCrd, mvccCntr, MVCC_OP_COUNTER_NA, false)) + return false; + + byte state = state(cctx, mvccCrd, mvccCntr, 0); + + return state != TxState.COMMITTED && state != TxState.ABORTED + || cctx.kernalContext().coordinators().hasLocalTransaction(mvccCrd, mvccCntr); + } + + /** + * @param cctx Cache context. + * @param mvccCrd Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @param mvccOpCntr Mvcc operation counter. + * @return TxState + * @see TxState + * @throws IgniteCheckedException If failed. + */ + public static byte state(GridCacheContext cctx, long mvccCrd, long mvccCntr, int mvccOpCntr) throws IgniteCheckedException { + return state(cctx.kernalContext().coordinators(), mvccCrd, mvccCntr, mvccOpCntr); + } + + /** + * @param grp Cache group context. + * @param mvccCrd Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @param mvccOpCntr Mvcc operation counter. + * @return TxState + * @see TxState + * @throws IgniteCheckedException If failed. + */ + public static byte state(CacheGroupContext grp, long mvccCrd, long mvccCntr, int mvccOpCntr) throws IgniteCheckedException { + return state(grp.shared().coordinators(), mvccCrd, mvccCntr, mvccOpCntr); + } + + /** + * @param proc Mvcc processor. + * @param mvccCrd Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @return TxState + * @see TxState + * @throws IgniteCheckedException If failed. + */ + private static byte state(MvccProcessor proc, long mvccCrd, long mvccCntr, int mvccOpCntr) throws IgniteCheckedException { + if (compare(INITIAL_VERSION, mvccCrd, mvccCntr, mvccOpCntr) == 0) + return TxState.COMMITTED; // Initial version is always committed; + + if ((mvccOpCntr & MVCC_HINTS_MASK) != 0) + return (byte)(mvccOpCntr >>> MVCC_HINTS_BIT_OFF); + + return proc.state(mvccCrd, mvccCntr); + } + + /** + * Checks if version is visible from the given snapshot. + * + * @param cctx Cache context. + * @param snapshot Snapshot. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + * @param opCntr Operation counter. + * @return {@code True} if visible. + * @throws IgniteCheckedException If failed. + */ + public static boolean isVisible(GridCacheContext cctx, MvccSnapshot snapshot, long mvccCrd, long mvccCntr, + int opCntr) throws IgniteCheckedException { + return isVisible(cctx, snapshot, mvccCrd, mvccCntr, opCntr, true); + } + + /** + * Checks if version is visible from the given snapshot. + * + * @param cctx Cache context. + * @param snapshot Snapshot. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + * @param opCntr Operation counter. + * @param useTxLog {@code True} if TxLog should be used. + * @return {@code True} if visible. + * @throws IgniteCheckedException If failed. + */ + public static boolean isVisible(GridCacheContext cctx, MvccSnapshot snapshot, long mvccCrd, long mvccCntr, + int opCntr, boolean useTxLog) throws IgniteCheckedException { + if (mvccCrd == MVCC_CRD_COUNTER_NA) { + assert mvccCntr == MVCC_COUNTER_NA && opCntr == MVCC_OP_COUNTER_NA + : "rowVer=" + mvccVersion(mvccCrd, mvccCntr, opCntr) + ", snapshot=" + snapshot; + + return false; // Unassigned version is always invisible + } + + if (compare(INITIAL_VERSION, mvccCrd, mvccCntr, opCntr) == 0) + return true; // Initial version is always visible + + long snapshotCrd = snapshot.coordinatorVersion(); + + long snapshotCntr = snapshot.counter(); + int snapshotOpCntr = snapshot.operationCounter(); + + if (mvccCrd > snapshotCrd) + return false; // Rows in the future are never visible. + + if (mvccCrd < snapshotCrd) + // Don't check the row with TxLog if the row is expected to be committed. + return !useTxLog || isCommitted(cctx, mvccCrd, mvccCntr, opCntr); + + if (mvccCntr > snapshotCntr) // we don't see future updates + return false; + + if (mvccCntr == snapshotCntr) { + assert opCntr <= snapshotOpCntr : "rowVer=" + mvccVersion(mvccCrd, mvccCntr, opCntr) + ", snapshot=" + snapshot; + + return opCntr < snapshotOpCntr; // we don't see own pending updates + } + + if (snapshot.activeTransactions().contains(mvccCntr)) // we don't see of other transactions' pending updates + return false; + + if (!useTxLog) + return true; // The checking row is expected to be committed. + + byte state = state(cctx, mvccCrd, mvccCntr, opCntr); + + if (state != TxState.COMMITTED && state != TxState.ABORTED) + throw unexpectedStateException(cctx, state, mvccCrd, mvccCntr, opCntr, snapshot); + + return state == TxState.COMMITTED; + } + + /** + * + * @param grp Cache group context. + * @param state State. + * @param crd Mvcc coordinator counter. + * @param cntr Mvcc counter. + * @param opCntr Mvcc operation counter. + * @return State exception. + */ + public static IgniteTxMvccVersionCheckedException unexpectedStateException( + CacheGroupContext grp, byte state, long crd, long cntr, + int opCntr) { + return unexpectedStateException(grp.shared().kernalContext(), state, crd, cntr, opCntr, null); + } + + /** + * + * @param cctx Cache context. + * @param state State. + * @param crd Mvcc coordinator counter. + * @param cntr Mvcc counter. + * @param opCntr Mvcc operation counter. + * @param snapshot Mvcc snapshot + * @return State exception. + */ + public static IgniteTxMvccVersionCheckedException unexpectedStateException( + GridCacheContext cctx, byte state, long crd, long cntr, + int opCntr, MvccSnapshot snapshot) { + return unexpectedStateException(cctx.kernalContext(), state, crd, cntr, opCntr, snapshot); + } + + /** */ + private static IgniteTxMvccVersionCheckedException unexpectedStateException(GridKernalContext ctx, byte state, long crd, long cntr, + int opCntr, MvccSnapshot snapshot) { + String msg = "Unexpected state: [state=" + state + ", rowVer=" + crd + ":" + cntr + ":" + opCntr; + + if (snapshot != null) + msg += ", txVer=" + snapshot.coordinatorVersion() + ":" + snapshot.counter() + ":" + snapshot.operationCounter(); + + msg += ", localNodeId=" + ctx.localNodeId() + "]"; + + return new IgniteTxMvccVersionCheckedException(msg); + } + + /** + * Checks visibility of the given row versions from the given snapshot. + * + * @param cctx Context. + * @param snapshot Snapshot. + * @param crd Mvcc coordinator counter. + * @param cntr Mvcc counter. + * @param opCntr Mvcc operation counter. + * @param link Link to data row (new version is located there). + * @return Visibility status. + * @throws IgniteCheckedException If failed. + */ + public static boolean isVisible(GridCacheContext cctx, MvccSnapshot snapshot, long crd, long cntr, + int opCntr, long link) throws IgniteCheckedException { + return isVisible(cctx, snapshot, crd, cntr, opCntr, false) + && isVisible(cctx, link, snapshot); + } + + /** + * Checks if a row has not empty new version (xid_max). + * + * @param row Row. + * @return {@code True} if row has a new version. + */ + public static boolean hasNewVersion(MvccUpdateVersionAware row) { + assert row.newMvccCoordinatorVersion() == MVCC_CRD_COUNTER_NA + || mvccVersionIsValid(row.newMvccCoordinatorVersion(), row.newMvccCounter(), row.newMvccOperationCounter()); + + return row.newMvccCoordinatorVersion() > MVCC_CRD_COUNTER_NA; + } + + /** + * Checks if a row's new version is visible for the given snapshot. + * + * @param cctx Cache context. + * @param link Link to the row. + * @param snapshot Mvcc snapshot. + * @return {@code True} if row is visible for the given snapshot. + * @throws IgniteCheckedException If failed. + */ + public static int getVisibleState(GridCacheContext cctx, long link, MvccSnapshot snapshot) + throws IgniteCheckedException { + return invoke(cctx, link, getVisibleState, snapshot); + } + + /** + * Returns new version of row (xid_max) if any. + * + * @param cctx Cache context. + * @param link Link to the row. + * @return New {@code MvccVersion} if row has xid_max, or null if doesn't. + * @throws IgniteCheckedException If failed. + */ + public static MvccVersion getNewVersion(GridCacheContext cctx, long link) + throws IgniteCheckedException { + return invoke(cctx, link, getNewVer, null); + } + + /** + * Compares row version (xid_min) with the given version. + * + * @param row Row. + * @param ver Version. + * @return Comparison result, see {@link Comparable}. + */ + public static int compare(MvccVersionAware row, MvccVersion ver) { + return compare(row.mvccCoordinatorVersion(), row.mvccCounter(), row.mvccOperationCounter(), + ver.coordinatorVersion(), ver.counter(), ver.operationCounter()); + } + + /** + * Compares to pairs of MVCC versions. See {@link Comparable}. + * + * @param mvccVerLeft First MVCC version. + * @param mvccCrdRight Second coordinator version. + * @param mvccCntrRight Second counter. + * @return Comparison result, see {@link Comparable}. + */ + public static int compare(MvccVersion mvccVerLeft, long mvccCrdRight, long mvccCntrRight) { + return compare(mvccVerLeft.coordinatorVersion(), mvccVerLeft.counter(), mvccCrdRight, mvccCntrRight); + } + + /** + * Compares to pairs of MVCC versions. See {@link Comparable}. + * + * @param row First MVCC version. + * @param mvccCrdRight Second coordinator version. + * @param mvccCntrRight Second counter. + * @return Comparison result, see {@link Comparable}. + */ + public static int compare(MvccVersionAware row, long mvccCrdRight, long mvccCntrRight) { + return compare(row.mvccCoordinatorVersion(), row.mvccCounter(), mvccCrdRight, mvccCntrRight); + } + + /** + * Compares to pairs of MVCC versions. See {@link Comparable}. + * + * @param mvccVerLeft First MVCC version. + * @param mvccCrdRight Second coordinator version. + * @param mvccCntrRight Second counter. + * @param mvccOpCntrRight Second operation counter. + * @return Comparison result, see {@link Comparable}. + */ + public static int compare(MvccVersion mvccVerLeft, long mvccCrdRight, long mvccCntrRight, int mvccOpCntrRight) { + return compare(mvccVerLeft.coordinatorVersion(), mvccVerLeft.counter(), + mvccVerLeft.operationCounter(), mvccCrdRight, mvccCntrRight, mvccOpCntrRight); + } + + /** + * Compares to pairs of coordinator/counter versions. See {@link Comparable}. + * + * @param mvccCrdLeft First coordinator version. + * @param mvccCntrLeft First counter version. + * @param mvccOpCntrLeft First operation counter. + * @param other The object to compare with. + * @return Comparison result, see {@link Comparable}. + */ + public static int compare(long mvccCrdLeft, long mvccCntrLeft, int mvccOpCntrLeft, MvccVersionAware other) { + return compare(mvccCrdLeft, mvccCntrLeft, mvccOpCntrLeft, + other.mvccCoordinatorVersion(), other.mvccCounter(), other.mvccOperationCounter()); + } + + /** + * Compares to pairs of coordinator/counter versions. See {@link Comparable}. + * + * @param mvccCrdLeft First coordinator version. + * @param mvccCntrLeft First counter version. + * @param mvccCrdRight Second coordinator version. + * @param mvccCntrRight Second counter version. + * @return Comparison result, see {@link Comparable}. + */ + public static int compare(long mvccCrdLeft, long mvccCntrLeft, long mvccCrdRight, long mvccCntrRight) { + return compare(mvccCrdLeft, mvccCntrLeft, 0, mvccCrdRight, mvccCntrRight, 0); + } + + /** + * Compares to pairs of coordinator/counter versions. See {@link Comparable}. + * + * @param mvccCrdLeft First coordinator version. + * @param mvccCntrLeft First counter version. + * @param mvccOpCntrLeft First operation counter. + * @param mvccCrdRight Second coordinator version. + * @param mvccCntrRight Second counter version. + * @param mvccOpCntrRight Second operation counter. + * @return Comparison result, see {@link Comparable}. + */ + public static int compare(long mvccCrdLeft, long mvccCntrLeft, int mvccOpCntrLeft, long mvccCrdRight, + long mvccCntrRight, int mvccOpCntrRight) { + int cmp; + + if ((cmp = Long.compare(mvccCrdLeft, mvccCrdRight)) != 0 + || (cmp = Long.compare(mvccCntrLeft, mvccCntrRight)) != 0 + || (cmp = Integer.compare(mvccOpCntrLeft & ~MVCC_HINTS_MASK, mvccOpCntrRight & ~MVCC_HINTS_MASK)) != 0) + return cmp; + + return 0; + } + + /** + * Compares new row version (xid_max) with the given counter and coordinator versions. + * + * @param row Row. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + * @return Comparison result, see {@link Comparable}. + */ + public static int compareNewVersion(MvccUpdateVersionAware row, long mvccCrd, long mvccCntr) { + return compare(row.newMvccCoordinatorVersion(), row.newMvccCounter(), mvccCrd, mvccCntr); + } + + /** + * Compares new row version (xid_max) with the given counter and coordinator versions. + * + * @param row Row. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + * @param opCntr Mvcc operation counter. + * @return Comparison result, see {@link Comparable}. + */ + public static int compareNewVersion(MvccUpdateVersionAware row, long mvccCrd, long mvccCntr, int opCntr) { + return compare(row.newMvccCoordinatorVersion(), row.newMvccCounter(), row.newMvccOperationCounter(), mvccCrd, mvccCntr, opCntr); + } + + /** + * Compares new row version (xid_max) with the given version. + * + * @param row Row. + * @param ver Version. + * @return Comparison result, see {@link Comparable}. + */ + public static int compareNewVersion(MvccUpdateVersionAware row, MvccVersion ver) { + return compare(row.newMvccCoordinatorVersion(), row.newMvccCounter(), row.newMvccOperationCounter(), + ver.coordinatorVersion(), ver.counter(), ver.operationCounter()); + } + + /** + * @param crdVer Mvcc coordinator version. + * @param cntr Counter. + * @param opCntr Operation counter. + * @return Always {@code true}. + */ + public static boolean mvccVersionIsValid(long crdVer, long cntr, int opCntr) { + return mvccVersionIsValid(crdVer, cntr) && opCntr != MVCC_OP_COUNTER_NA; + } + + /** + * @param crdVer Mvcc coordinator version. + * @param cntr Counter. + * @return {@code True} if version is valid. + */ + public static boolean mvccVersionIsValid(long crdVer, long cntr) { + return crdVer > MVCC_CRD_COUNTER_NA && cntr != MVCC_COUNTER_NA; + } + + /** + * @param topVer Topology version for cache operation. + * @return Error. + */ + public static ClusterTopologyServerNotFoundException noCoordinatorError(AffinityTopologyVersion topVer) { + return new ClusterTopologyServerNotFoundException("Mvcc coordinator is not assigned for " + + "topology version: " + topVer); + } + + /** + * @return Error. + */ + public static ClusterTopologyServerNotFoundException noCoordinatorError() { + return new ClusterTopologyServerNotFoundException("Mvcc coordinator is not assigned."); + } + + /** + * @param cctx Cache context. + * @param link Link to the row. + * @param snapshot Mvcc snapshot. + * @return {@code True} if row is updated for given snapshot. + * @throws IgniteCheckedException If failed. + */ + private static boolean isVisible(GridCacheContext cctx, long link, + MvccSnapshot snapshot) + throws IgniteCheckedException { + return invoke(cctx, link, isVisible, snapshot); + } + + /** + * Encapsulates common logic for working with row mvcc info: page locking/unlocking, checks and other. + * Strategy pattern. + * + * @param cctx Cache group. + * @param link Row link. + * @param clo Closure to apply. + * @param snapshot Mvcc snapshot. + * @param Return type. + * @return Result. + * @throws IgniteCheckedException If failed. + */ + private static R invoke(GridCacheContext cctx, long link, MvccClosure clo, MvccSnapshot snapshot) + throws IgniteCheckedException { + assert cctx.mvccEnabled(); + + PageMemory pageMem = cctx.dataRegion().pageMemory(); + int grpId = cctx.groupId(); + + long pageId = pageId(link); + long page = pageMem.acquirePage(grpId, pageId); + + try { + long pageAddr = pageMem.readLock(grpId, pageId, page); + + try{ + DataPageIO dataIo = DataPageIO.VERSIONS.forPage(pageAddr); + + int offset = dataIo.getPayloadOffset(pageAddr, itemId(link), pageMem.pageSize(), MVCC_INFO_SIZE); + + long mvccCrd = dataIo.mvccCoordinator(pageAddr, offset); + long mvccCntr = dataIo.mvccCounter(pageAddr, offset); + int mvccOpCntr = dataIo.mvccOperationCounter(pageAddr, offset); + + assert mvccVersionIsValid(mvccCrd, mvccCntr, mvccOpCntr) : mvccVersion(mvccCrd, mvccCntr, mvccOpCntr); + + long newMvccCrd = dataIo.newMvccCoordinator(pageAddr, offset); + long newMvccCntr = dataIo.newMvccCounter(pageAddr, offset); + int newMvccOpCntr = dataIo.newMvccOperationCounter(pageAddr, offset); + + assert newMvccCrd == MVCC_CRD_COUNTER_NA || mvccVersionIsValid(newMvccCrd, newMvccCntr, newMvccOpCntr) + : mvccVersion(newMvccCrd, newMvccCntr, newMvccOpCntr); + + return clo.apply(cctx, snapshot, mvccCrd, mvccCntr, mvccOpCntr, newMvccCrd, newMvccCntr, newMvccOpCntr); + } + finally { + pageMem.readUnlock(grpId, pageId, page); + } + } + finally { + pageMem.releasePage(grpId, pageId, page); + } + } + + /** + * + * @param cctx Cache context. + * @param mvccCrd Coordinator version. + * @param mvccCntr Counter. + * @return {@code True} in case the corresponding transaction is in {@code TxState.COMMITTED} state. + * @throws IgniteCheckedException If failed. + */ + private static boolean isCommitted(GridCacheContext cctx, long mvccCrd, long mvccCntr, int mvccOpCntr) throws IgniteCheckedException { + return state(cctx, mvccCrd, mvccCntr, mvccOpCntr) == TxState.COMMITTED; + } + + /** + * Throw an {@link UnsupportedOperationException} if this cache is transactional and MVCC is enabled with + * appropriate message about corresponding operation type. + * @param cctx Cache context. + * @param opType operation type to mention in error message. + */ + public static void verifyMvccOperationSupport(GridCacheContext cctx, String opType) { + if (cctx.mvccEnabled()) + throw new UnsupportedOperationException(opType + " operations are not supported on transactional " + + "caches when MVCC is enabled."); + } + + /** + * Checks transaction state. + * @param tx Transaction. + * @return Checked transaction. + */ + public static GridNearTxLocal checkActive(GridNearTxLocal tx) { + if (tx != null && tx.state() != TransactionState.ACTIVE) + throw new IgniteSQLException("Transaction is already completed.", TRANSACTION_COMPLETED); + + return tx; + } + + + /** + * @param ctx Grid kernal context. + * @return Currently started user transaction, or {@code null} if none started. + */ + @Nullable public static GridNearTxLocal tx(GridKernalContext ctx) { + return tx(ctx, null); + } + + /** + * @param ctx Grid kernal context. + * @param txId Transaction ID. + * @return Currently started user transaction, or {@code null} if none started. + */ + @Nullable public static GridNearTxLocal tx(GridKernalContext ctx, @Nullable GridCacheVersion txId) { + IgniteTxManager tm = ctx.cache().context().tm(); + + IgniteInternalTx tx0 = txId == null ? tm.tx() : tm.tx(txId); + + GridNearTxLocal tx = tx0 != null && tx0.user() ? (GridNearTxLocal)tx0 : null; + + if (tx != null) { + if (!tx.pessimistic() || !tx.repeatableRead()) { + tx.setRollbackOnly(); + + throw new IgniteSQLException("Only pessimistic repeatable read transactions are supported at the moment.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + } + + if (!tx.isOperationAllowed(true)) { + tx.setRollbackOnly(); + + throw new IgniteSQLException("SQL queries and cache operations " + + "may not be used in the same transaction.", IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH); + } + } + + return tx; + } + + + /** + * @param ctx Grid kernal context. + * @param timeout Transaction timeout. + * @return Newly started SQL transaction. + */ + public static GridNearTxLocal txStart(GridKernalContext ctx, long timeout) { + return txStart(ctx, null, timeout); + } + + /** + * @param cctx Cache context. + * @param timeout Transaction timeout. + * @return Newly started SQL transaction. + */ + public static GridNearTxLocal txStart(GridCacheContext cctx, long timeout) { + return txStart(cctx.kernalContext(), cctx, timeout); + } + + /** + * @param ctx Grid kernal context. + * @param cctx Cache context. + * @param timeout Transaction timeout. + * @return Newly started SQL transaction. + */ + private static GridNearTxLocal txStart(GridKernalContext ctx, @Nullable GridCacheContext cctx, long timeout) { + if (timeout == 0) { + TransactionConfiguration tcfg = cctx != null ? + CU.transactionConfiguration(cctx, ctx.config()) : null; + + if (tcfg != null) + timeout = tcfg.getDefaultTxTimeout(); + } + + GridNearTxLocal tx = ctx.cache().context().tm().newTx( + false, + false, + cctx != null && cctx.systemTx() ? cctx : null, + PESSIMISTIC, + REPEATABLE_READ, + timeout, + cctx == null || !cctx.skipStore(), + true, + 0, + null + ); + + tx.syncMode(FULL_SYNC); + + return tx; + } + + /** + * @param ctx Grid kernal context. + * @return Whether MVCC is enabled or not on {@link IgniteConfiguration}. + */ + public static boolean mvccEnabled(GridKernalContext ctx) { + return ctx.config().isMvccEnabled(); + } + + /** + * Initialises MVCC filter and returns MVCC query tracker if needed. + * @param cctx Cache context. + * @param startTx Start transaction flag. + * @return MVCC query tracker. + * @throws IgniteCheckedException If failed. + */ + @NotNull public static MvccQueryTracker mvccTracker(GridCacheContext cctx, boolean startTx) throws IgniteCheckedException { + assert cctx != null && cctx.mvccEnabled(); + + GridNearTxLocal tx = tx(cctx.kernalContext()); + + if (tx == null && startTx) + tx = txStart(cctx, 0); + + return mvccTracker(cctx, tx); + } + + /** + * Initialises MVCC filter and returns MVCC query tracker if needed. + * @param cctx Cache context. + * @param tx Transaction. + * @return MVCC query tracker. + * @throws IgniteCheckedException If failed. + */ + @NotNull public static MvccQueryTracker mvccTracker(GridCacheContext cctx, + GridNearTxLocal tx) throws IgniteCheckedException { + MvccQueryTracker tracker; + + if (tx == null) + tracker = new MvccQueryTrackerImpl(cctx); + else if ((tracker = tx.mvccQueryTracker()) == null) + tracker = new StaticMvccQueryTracker(cctx, requestSnapshot(cctx, tx)) { + @Override public void onDone() { + // TODO IGNITE-8841 + checkActive(tx); + } + }; + + if (tracker.snapshot() == null) + // TODO IGNITE-7388 + tracker.requestSnapshot().get(); + + return tracker; + } + + /** + * @param cctx Cache context. + * @param tx Transaction. + * @throws IgniteCheckedException If failed. + * @return Mvcc snapshot. + */ + public static MvccSnapshot requestSnapshot(GridCacheContext cctx, + GridNearTxLocal tx) throws IgniteCheckedException { + MvccSnapshot snapshot; tx = checkActive(tx); + + if ((snapshot = tx.mvccSnapshot()) == null) { + MvccProcessor prc = cctx.shared().coordinators(); + + snapshot = prc.tryRequestSnapshotLocal(tx); + + if (snapshot == null) + // TODO IGNITE-7388 + snapshot = prc.requestSnapshotAsync(tx).get(); + + tx.mvccSnapshot(snapshot); + } + + return snapshot; + } + + /** */ + private static MvccVersion mvccVersion(long crd, long cntr, int opCntr) { + return new MvccVersionImpl(crd, cntr, opCntr); + } + + /** + * Mvcc closure interface. + * @param Return type. + */ + private interface MvccClosure { + /** + * Runs closure over the Mvcc info. + * @param snapshot Mvcc snapshot. + * @param mvccCrd Coordinator version. + * @param mvccCntr Counter. + * @param mvccOpCntr Operation counter. + * @param newMvccCrd New mvcc coordinator + * @param newMvccCntr New mvcc counter. + * @param newMvccOpCntr New mvcc operation counter. + * @return Result. + */ + public R apply(GridCacheContext cctx, MvccSnapshot snapshot, long mvccCrd, long mvccCntr, int mvccOpCntr, + long newMvccCrd, long newMvccCntr, int newMvccOpCntr) throws IgniteCheckedException; + } + + /** + * Closure for checking row visibility for snapshot. + */ + private static class GetVisibleState implements MvccClosure { + /** {@inheritDoc} */ + @Override public Integer apply(GridCacheContext cctx, MvccSnapshot snapshot, long mvccCrd, long mvccCntr, + int mvccOpCntr, long newMvccCrd, long newMvccCntr, int newMvccOpCntr) throws IgniteCheckedException { + + if (!isVisible(cctx, snapshot, mvccCrd, mvccCntr, mvccOpCntr)) + return MVCC_INVISIBLE; + + if (newMvccCrd == MVCC_CRD_COUNTER_NA) + return MVCC_VISIBLE; + + assert mvccVersionIsValid(newMvccCrd, newMvccCntr, newMvccOpCntr); + + if (mvccCrd == newMvccCrd && mvccCntr == newMvccCntr) // Double-changed in scope of one transaction. + return MVCC_VISIBLE_REMOVED; + + return isVisible(cctx, snapshot, newMvccCrd, newMvccCntr, newMvccOpCntr) ? MVCC_VISIBLE_REMOVED : + MVCC_VISIBLE; + } + } + + /** + * Closure for checking whether the row is visible for given snapshot. + */ + private static class IsVisible implements MvccClosure { + /** {@inheritDoc} */ + @Override public Boolean apply(GridCacheContext cctx, MvccSnapshot snapshot, long mvccCrd, long mvccCntr, + int mvccOpCntr, long newMvccCrd, long newMvccCntr, int newMvccOpCntr) throws IgniteCheckedException { + + if (!isVisible(cctx, snapshot, mvccCrd, mvccCntr, mvccOpCntr)) + return false; + + if (newMvccCrd == MVCC_CRD_COUNTER_NA) + return true; + + assert mvccVersionIsValid(newMvccCrd, newMvccCntr, newMvccOpCntr); + + if (mvccCrd == newMvccCrd && mvccCntr == newMvccCntr) // Double-changed in scope of one transaction. + return false; + + return !isVisible(cctx, snapshot, newMvccCrd, newMvccCntr, newMvccOpCntr); + } + } + + /** + * Closure for getting xid_max version of row. + */ + private static class GetNewVersion implements MvccClosure { + /** {@inheritDoc} */ + @Override public MvccVersion apply(GridCacheContext cctx, MvccSnapshot snapshot, long mvccCrd, long mvccCntr, + int mvccOpCntr, long newMvccCrd, long newMvccCntr, int newMvccOpCntr) { + return newMvccCrd == MVCC_CRD_COUNTER_NA ? null : mvccVersion(newMvccCrd, newMvccCntr, newMvccOpCntr); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersion.java new file mode 100644 index 0000000000000..f43d3b9b93641 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersion.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.jetbrains.annotations.NotNull; + +/** + * MVCC version. This is unique version allowing to order all reads and writes within a cluster. Consists of two parts: + * - coordinator version - number which increases on every coordinator change; + * - counter - local coordinator counter which is increased on every update. + */ +public interface MvccVersion extends Comparable { + /** + * @return Coordinator version. + */ + public long coordinatorVersion(); + + /** + * @return Local counter. + */ + public long counter(); + + /** + * @return Operation id in scope of current transaction. + */ + public int operationCounter(); + + /** {@inheritDoc} */ + @Override default int compareTo(@NotNull MvccVersion another) { + return MvccUtils.compare(coordinatorVersion(), counter(), operationCounter(), + another.coordinatorVersion(), another.counter(), another.operationCounter()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionAware.java new file mode 100644 index 0000000000000..3bfefbc365997 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionAware.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.processors.cache.mvcc; + +/** + * + */ +public interface MvccVersionAware { + /** + * @return Mvcc coordinator version. + */ + public long mvccCoordinatorVersion(); + + /** + * @return Mvcc counter. + */ + public long mvccCounter(); + + /** + * @return Mvcc operation counter. + */ + public int mvccOperationCounter(); + + /** + * @return Tx state hint for 'created' mvcc version. + */ + public byte mvccTxState(); + + /** + * Copies mvcc version from another object. + * @param other Info source. + */ + public default void mvccVersion(MvccVersionAware other) { + mvccVersion(other.mvccCoordinatorVersion(), other.mvccCounter(), other.mvccOperationCounter()); + } + + /** + * Sets mvcc version. + * @param ver Mvcc version. + */ + public default void mvccVersion(MvccVersion ver) { + mvccVersion(ver.coordinatorVersion(), ver.counter(), ver.operationCounter()); + } + + /** + * Sets mvcc version. + * @param crd Mvcc coordinator version. + * @param cntr Mvcc counter. + * @param opCntr Mvcc operation counter. + */ + public default void mvccVersion(long crd, long cntr, int opCntr) { + throw new UnsupportedOperationException(); + } + + /** + * @return Mvcc version. + */ + public default MvccVersion mvccVersion() { + return new MvccVersionImpl(mvccCoordinatorVersion(), mvccCounter(), mvccOperationCounter()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionImpl.java new file mode 100644 index 0000000000000..ec3e1370d06cf --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccVersionImpl.java @@ -0,0 +1,193 @@ +/* + * 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.mvcc; + +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +import java.nio.ByteBuffer; + +/** + * Base MVCC version implementation. + */ +public class MvccVersionImpl implements MvccVersion, Message { + /** */ + private static final long serialVersionUID = 0L; + + /** Coordinator version. */ + private long crdVer; + + /** Local counter. */ + private long cntr; + + /** Operation counter. */ + private int opCntr; + + /** + * Constructor. + */ + public MvccVersionImpl() { + // No-op. + } + + /** + * @param crdVer Coordinator version. + * @param cntr Counter. + * @param opCntr Operation counter. + */ + public MvccVersionImpl(long crdVer, long cntr, int opCntr) { + this.crdVer = crdVer; + this.cntr = cntr; + this.opCntr = opCntr; + } + + /** + * @return Coordinator version. + */ + public long coordinatorVersion() { + return crdVer; + } + + /** + * @return Local counter. + */ + public long counter() { + return cntr; + } + + /** {@inheritDoc} */ + @Override public int operationCounter() { + return opCntr; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + MvccVersionImpl that = (MvccVersionImpl) o; + + return crdVer == that.crdVer && cntr == that.cntr; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = (int) (crdVer ^ (crdVer >>> 32)); + + res = 31 * res + (int) (cntr ^ (cntr >>> 32)); + + return res; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("cntr", cntr)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeLong("crdVer", crdVer)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeInt("opCntr", opCntr)) + 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: + cntr = reader.readLong("cntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + crdVer = reader.readLong("crdVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + opCntr = reader.readInt("opCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccVersionImpl.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 148; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 3; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccVersionImpl.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/NoOpMvccProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/NoOpMvccProcessor.java new file mode 100644 index 0000000000000..b9a513296aca8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/NoOpMvccProcessor.java @@ -0,0 +1,215 @@ +/* + * 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.mvcc; + +import java.util.Collection; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteDiagnosticPrepareContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.ExchangeContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.util.GridLongList; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +class NoOpMvccProcessor extends GridProcessorAdapter implements MvccProcessor { + /** + * @param ctx Kernal context. + */ + protected NoOpMvccProcessor(GridKernalContext ctx) { + super(ctx); + } + + /** {@inheritDoc} */ + @Override public void onDiscoveryEvent(int evtType, Collection nodes, long topVer) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onExchangeStart(MvccCoordinator mvccCrd, ExchangeContext exchCtx, ClusterNode exchCrd) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onExchangeDone(boolean newCoord, DiscoCache discoCache, + Map activeQueries) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void processClientActiveQueries(UUID nodeId, @Nullable GridLongList activeQueries) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Nullable @Override public MvccCoordinator currentCoordinator() { + return null; + } + + /** {@inheritDoc} */ + @Nullable @Override public MvccCoordinator currentCoordinator(AffinityTopologyVersion topVer) { + return null; + } + + /** {@inheritDoc} */ + @Nullable @Override public MvccCoordinator coordinatorFromDiscoveryEvent() { + return null; + } + + /** {@inheritDoc} */ + @Override public UUID currentCoordinatorId() { + return null; + } + + /** {@inheritDoc} */ + @Override public void updateCoordinator(MvccCoordinator curCrd) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public byte state(long crdVer, long cntr) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public byte state(MvccVersion ver) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void updateState(MvccVersion ver, byte state) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void updateState(MvccVersion ver, byte state, boolean primary) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void registerLocalTransaction(long crd, long cntr) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public boolean hasLocalTransaction(long crd, long cntr) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture waitFor(GridCacheContext cctx, + MvccVersion locked) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void addQueryTracker(MvccQueryTracker tracker) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void removeQueryTracker(Long id) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot tryRequestSnapshotLocal() { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot tryRequestSnapshotLocal( + @Nullable IgniteInternalTx tx) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshotAsync() { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshotAsync(IgniteInternalTx tx) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void requestSnapshotAsync(MvccSnapshotResponseListener lsnr) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void requestSnapshotAsync(IgniteInternalTx tx, MvccSnapshotResponseListener lsnr) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture ackTxCommit(MvccSnapshot updateVer) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture ackTxCommit(MvccVersion updateVer, MvccSnapshot readSnapshot, + long qryId) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void ackTxRollback(MvccVersion updateVer) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void ackTxRollback(MvccVersion updateVer, MvccSnapshot readSnapshot, long qryTrackerId) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void ackQueryDone(MvccSnapshot snapshot, long qryId) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture waitTxsFuture(UUID crdId, GridLongList txs) { + throw processorException(); + } + + /** {@inheritDoc} */ + @Override public void dumpDebugInfo(IgniteLogger log, @Nullable IgniteDiagnosticPrepareContext diagCtx) { + // No-op. + } + + /** + * @return No-op processor usage exception; + */ + private IgniteException processorException() { + return new IgniteException("Current Ignite configuration does not support MVCC functionality " + + "(consider adding ignite-schedule module to classpath)."); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/StaticMvccQueryTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/StaticMvccQueryTracker.java new file mode 100644 index 0000000000000..52fb1db6e63e8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/StaticMvccQueryTracker.java @@ -0,0 +1,96 @@ +/* + * 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.mvcc; + +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.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.jetbrains.annotations.NotNull; + +/** + * Simple MVCC tracker used only as an Mvcc snapshot holder. + */ +public class StaticMvccQueryTracker implements MvccQueryTracker { + /** */ + private final MvccSnapshot snapshot; + /** */ + private final GridCacheContext cctx; + + /** + * @param cctx Cache context. + * @param snapshot Mvcc snapshot. + */ + public StaticMvccQueryTracker(GridCacheContext cctx, MvccSnapshot snapshot) { + this.snapshot = snapshot; + this.cctx = cctx; + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot snapshot() { + assert snapshot != null : this; + + return snapshot; + } + + /** {@inheritDoc} */ + @Override public GridCacheContext context() { + return cctx; + } + + /** {@inheritDoc} */ + @Override public AffinityTopologyVersion topologyVersion() { + return AffinityTopologyVersion.NONE; + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshot() { + return new GridFinishedFuture<>(snapshot); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture requestSnapshot(@NotNull final AffinityTopologyVersion topVer) { + return new GridFinishedFuture<>(snapshot); + } + + /** {@inheritDoc} */ + @Override public void requestSnapshot(@NotNull AffinityTopologyVersion topVer, @NotNull MvccSnapshotResponseListener lsnr) { + lsnr.onResponse(snapshot); + } + + /** {@inheritDoc} */ + @Override public void onDone() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture onDone(@NotNull GridNearTxLocal tx, boolean commit) { + throw new UnsupportedOperationException("Operation is not supported."); + } + + /** {@inheritDoc} */ + @Override public long onMvccCoordinatorChange(MvccCoordinator newCrd) { + return MVCC_TRACKER_ID_NA; + } + + /** {@inheritDoc} */ + @Override public long id() { + return MVCC_TRACKER_ID_NA; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetrics.java new file mode 100644 index 0000000000000..1de297f9e88f5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetrics.java @@ -0,0 +1,102 @@ +/* + * 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.mvcc; + +/** + * + */ +public class VacuumMetrics { + /** */ + private long cleanupRowsCnt; + + /** */ + private long scannedRowsCnt; + + /** */ + private long searchNanoTime; + + /** */ + private long cleanupNanoTime; + + /** + * @return Cleanup rows count. + */ + public long cleanupRowsCount() { + return cleanupRowsCnt; + } + + /** + * @return Scanned rows count. + */ + public long scannedRowsCount() { + return scannedRowsCnt; + } + + /** + * @return Search nano time. + */ + public long searchNanoTime() { + return searchNanoTime; + } + + /** + * @return Cleanup nano time + */ + public long cleanupNanoTime() { + return cleanupNanoTime; + } + + + /** + * @param delta Delta. + */ + public void addCleanupRowsCnt(long delta) { + cleanupRowsCnt += delta; + } + + /** + * @param delta Delta. + */ + public void addScannedRowsCount(long delta) { + scannedRowsCnt += delta; + } + + /** + * @param delta Delta. + */ + public void addSearchNanoTime(long delta) { + searchNanoTime += delta; + } + + /** + * @param delta Delta. + */ + public void addCleanupNanoTime(long delta) { + cleanupNanoTime += delta; + } + + /** */ + @Override public String toString() { + return "VacuumMetrics[" + + "cleanupRowsCnt=" + cleanupRowsCnt + + ", scannedRowsCnt=" + scannedRowsCnt + + ", searchNanoTime=" + Math.round((float)searchNanoTime / 1_000_000) + + " ms, cleanupNanoTime=" + Math.round((float)cleanupNanoTime / 1_000_000) + + " ms]"; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetricsReducer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetricsReducer.java new file mode 100644 index 0000000000000..c952a48ad1da4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumMetricsReducer.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.processors.cache.mvcc; + +import org.apache.ignite.lang.IgniteReducer; +import org.jetbrains.annotations.Nullable; + +/** + * Vacuum metrics reducer. + */ +public class VacuumMetricsReducer implements IgniteReducer { + /** */ + private static final long serialVersionUID = 7063457745963917386L; + + /** */ + private final VacuumMetrics m = new VacuumMetrics(); + + /** {@inheritDoc} */ + @Override public boolean collect(@Nullable VacuumMetrics metrics) { + assert metrics != null; + + m.addCleanupRowsCnt(metrics.cleanupRowsCount()); + m.addScannedRowsCount(metrics.scannedRowsCount()); + m.addSearchNanoTime(metrics.searchNanoTime()); + m.addCleanupNanoTime(metrics.cleanupNanoTime()); + + return true; + } + + /** {@inheritDoc} */ + @Override public VacuumMetrics reduce() { + return m; + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumTask.java new file mode 100644 index 0000000000000..9a0d9e2d644c5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/VacuumTask.java @@ -0,0 +1,63 @@ +/* + * 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.mvcc; + +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Task for cleaning sing partition. + */ +public class VacuumTask extends GridFutureAdapter { + /** */ + private final MvccSnapshot snapshot; + + /** */ + @GridToStringExclude + private final GridDhtLocalPartition part; + + /** + * @param snapshot Snapshot. + * @param part Partition to cleanup. + */ + VacuumTask(MvccSnapshot snapshot, GridDhtLocalPartition part) { + this.snapshot = snapshot; + this.part = part; + } + + /** + * @return Snapshot. + */ + public MvccSnapshot snapshot() { + return snapshot; + } + + /** + * @return Partition to cleanup. + */ + public GridDhtLocalPartition part() { + return part; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VacuumTask.class, this, "partId", part.id()); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryCntr.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryCntr.java new file mode 100644 index 0000000000000..0156c535f5b20 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryCntr.java @@ -0,0 +1,130 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * + */ +public class MvccAckRequestQueryCntr implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long cntr; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccAckRequestQueryCntr() { + // No-op. + } + + /** + * @param cntr Query counter. + */ + public MvccAckRequestQueryCntr(long cntr) { + this.cntr = cntr; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return true; + } + + /** + * @return Counter. + */ + public long counter() { + return cntr; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("cntr", cntr)) + 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: + cntr = reader.readLong("cntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccAckRequestQueryCntr.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 140; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccAckRequestQueryCntr.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryId.java new file mode 100644 index 0000000000000..7771f4d57915f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestQueryId.java @@ -0,0 +1,130 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * + */ +public class MvccAckRequestQueryId implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long qryTrackerId; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccAckRequestQueryId() { + // No-op. + } + + /** + * @param qryTrackerId Query tracker Id. + */ + public MvccAckRequestQueryId(long qryTrackerId) { + this.qryTrackerId = qryTrackerId; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return true; + } + + /** + * @return Query tracker id. + */ + public long queryTrackerId() { + return qryTrackerId; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("qryTrackerId", qryTrackerId)) + 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: + qryTrackerId = reader.readLong("qryTrackerId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccAckRequestQueryId.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 145; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccAckRequestQueryId.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTx.java new file mode 100644 index 0000000000000..69dfd254c73cc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTx.java @@ -0,0 +1,210 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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 static org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker.MVCC_TRACKER_ID_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; + +/** + * + */ +public class MvccAckRequestTx implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private static final int SKIP_RESPONSE_FLAG_MASK = 0x01; + + /** */ + private long futId; + + /** */ + private long txCntr; + + /** */ + private byte flags; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccAckRequestTx() { + // No-op. + } + + /** + * @param futId Future ID. + * @param txCntr Counter assigned to transaction. + */ + public MvccAckRequestTx(long futId, long txCntr) { + this.futId = futId; + this.txCntr = txCntr; + } + + /** + * @return Query counter. + */ + public long queryCounter() { + return MVCC_COUNTER_NA; + } + + /** + * @return Query tracker id. + */ + public long queryTrackerId() { + return MVCC_TRACKER_ID_NA; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return true; + } + + /** + * @return Future ID. + */ + public long futureId() { + return futId; + } + + /** + * @return {@code True} if response message is not needed. + */ + public boolean skipResponse() { + return (flags & SKIP_RESPONSE_FLAG_MASK) != 0; + } + + /** + * @param val {@code True} if response message is not needed. + */ + public void skipResponse(boolean val) { + if (val) + flags |= SKIP_RESPONSE_FLAG_MASK; + else + flags &= ~SKIP_RESPONSE_FLAG_MASK; + } + + /** + * @return Counter assigned tp transaction. + */ + public long txCounter() { + return txCntr; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeByte("flags", flags)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeLong("futId", futId)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeLong("txCntr", txCntr)) + 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: + futId = reader.readLong("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + txCntr = reader.readLong("txCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccAckRequestTx.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 137; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 3; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccAckRequestTx.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryCntr.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryCntr.java new file mode 100644 index 0000000000000..99761c3f72dc3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryCntr.java @@ -0,0 +1,123 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * + */ +public class MvccAckRequestTxAndQueryCntr extends MvccAckRequestTx { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long qryCntr; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccAckRequestTxAndQueryCntr() { + // No-op. + } + + /** + * @param futId Future ID. + * @param txCntr Counter assigned to transaction update. + * @param qryCntr Counter assigned for transaction reads. + */ + public MvccAckRequestTxAndQueryCntr(long futId, long txCntr, long qryCntr) { + super(futId, txCntr); + + this.qryCntr = qryCntr; + } + + /** {@inheritDoc} */ + @Override public long queryCounter() { + return qryCntr; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeLong("qryCntr", qryCntr)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + qryCntr = reader.readLong("qryCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccAckRequestTxAndQueryCntr.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 146; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 4; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccAckRequestTxAndQueryCntr.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java new file mode 100644 index 0000000000000..89f09db5d0b2f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccAckRequestTxAndQueryId.java @@ -0,0 +1,122 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * + */ +public class MvccAckRequestTxAndQueryId extends MvccAckRequestTx { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long qryTrackerId; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccAckRequestTxAndQueryId() { + // No-op. + } + + /** + * @param futId Future ID. + * @param txCntr Counter assigned to transaction update. + * @param qryTrackerId Query tracker id. + */ + public MvccAckRequestTxAndQueryId(long futId, long txCntr, long qryTrackerId) { + super(futId, txCntr); + + this.qryTrackerId = qryTrackerId; + } + + /** {@inheritDoc} */ + @Override public long queryTrackerId() { + return qryTrackerId; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeLong("qryTrackerId", qryTrackerId)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + qryTrackerId = reader.readLong("qryTrackerId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccAckRequestTxAndQueryId.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 147; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 4; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccAckRequestTxAndQueryId.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccActiveQueriesMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccActiveQueriesMessage.java new file mode 100644 index 0000000000000..4b78c24585f44 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccActiveQueriesMessage.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.util.GridLongList; +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.Nullable; + +/** + * + */ +public class MvccActiveQueriesMessage implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private GridLongList activeQrys; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccActiveQueriesMessage() { + // No-op. + } + + /** + * @param activeQrys Active queries. + */ + public MvccActiveQueriesMessage(GridLongList activeQrys) { + this.activeQrys = activeQrys; + } + + /** + * @return Active queries. + */ + @Nullable public GridLongList activeQueries() { + return activeQrys; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeMessage("activeQrys", activeQrys)) + 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: + activeQrys = reader.readMessage("activeQrys"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccActiveQueriesMessage.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 149; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccActiveQueriesMessage.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccFutureResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccFutureResponse.java new file mode 100644 index 0000000000000..72e4c52061d27 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccFutureResponse.java @@ -0,0 +1,130 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * + */ +public class MvccFutureResponse implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long futId; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccFutureResponse() { + // No-op. + } + + /** + * @param futId Future ID. + */ + public MvccFutureResponse(long futId) { + this.futId = futId; + } + + /** + * @return Future ID. + */ + public long futureId() { + return futId; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("futId", futId)) + 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: + futId = reader.readLong("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccFutureResponse.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 138; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccFutureResponse.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccMessage.java new file mode 100644 index 0000000000000..6d8b3c4042501 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccMessage.java @@ -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. + */ + +package org.apache.ignite.internal.processors.cache.mvcc.msg; + +import org.apache.ignite.plugin.extensions.communication.Message; + +/** + * Common interface for all MVCC-related messages. + */ +public interface MvccMessage extends Message { + /** + * @return {@code True} if should wait for coordinator initialization. + */ + public boolean waitForCoordinatorInit(); + + /** + * @return {@code True} if message should be processed from NIO thread. + */ + public boolean processedFromNioThread(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccQuerySnapshotRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccQuerySnapshotRequest.java new file mode 100644 index 0000000000000..75d33a7213c14 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccQuerySnapshotRequest.java @@ -0,0 +1,130 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * Request to get MVCC snapshot for a query. + */ +public class MvccQuerySnapshotRequest implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long futId; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccQuerySnapshotRequest() { + // No-op. + } + + /** + * @param futId Future ID. + */ + public MvccQuerySnapshotRequest(long futId) { + this.futId = futId; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return true; + } + + /** + * @return Future ID. + */ + public long futureId() { + return futId; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("futId", futId)) + 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: + futId = reader.readLong("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccQuerySnapshotRequest.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 139; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccQuerySnapshotRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java new file mode 100644 index 0000000000000..196003c8c3fa8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccSnapshotResponse.java @@ -0,0 +1,320 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.processors.cache.mvcc.MvccLongList; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshotWithoutTxs; +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; + +/** + * + */ +public class MvccSnapshotResponse implements MvccMessage, MvccSnapshot, MvccLongList { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long futId; + + /** */ + private long crdVer; + + /** */ + private long cntr; + + /** */ + private int opCntr; + + /** */ + @GridDirectTransient + private int txsCnt; + + /** */ + private long[] txs; + + /** */ + private long cleanupVer; + + /** */ + @GridDirectTransient + private long tracking; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccSnapshotResponse() { + // No-op. + } + + /** + * @param futId Future ID. + * @param crdVer Coordinator version. + * @param cntr Counter. + * @param opCntr Operation counter. + * @param cleanupVer Cleanup version. + * @param tracking Tracking number. + */ + public void init(long futId, long crdVer, long cntr, int opCntr, long cleanupVer, long tracking) { + this.futId = futId; + this.crdVer = crdVer; + this.cntr = cntr; + this.opCntr = opCntr; + this.cleanupVer = cleanupVer; + this.tracking = tracking; + + if (txsCnt > 0 && txs.length > txsCnt) // truncate if necessary + txs = Arrays.copyOf(txs, txsCnt); + } + + /** + * @param txId Transaction counter. + */ + public void addTx(long txId) { + if (txs == null) + txs = new long[4]; + else if (txs.length == txsCnt) + txs = Arrays.copyOf(txs, txs.length << 1); + + txs[txsCnt++] = txId; + } + + /** {@inheritDoc} */ + @Override public int size() { + return txsCnt; + } + + /** {@inheritDoc} */ + @Override public long get(int i) { + return txs[i]; + } + + /** {@inheritDoc} */ + @Override public boolean contains(long val) { + for (int i = 0; i < txsCnt; i++) { + if (txs[i] == val) + return true; + } + + return false; + } + + /** + * @return Tracking counter. + */ + public long tracking() { + return tracking; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return false; + } + + /** + * @return Future ID. + */ + public long futureId() { + return futId; + } + + /** {@inheritDoc} */ + @Override public long cleanupVersion() { + return cleanupVer; + } + + /** {@inheritDoc} */ + public long counter() { + return cntr; + } + + /** {@inheritDoc} */ + @Override public int operationCounter() { + return opCntr; + } + + /** {@inheritDoc} */ + @Override public void incrementOperationCounter() { + opCntr++; + } + + /** {@inheritDoc} */ + @Override public MvccLongList activeTransactions() { + return this; + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot withoutActiveTransactions() { + if (txsCnt > 0) + return new MvccSnapshotWithoutTxs(crdVer, cntr, opCntr, cleanupVer); + + return this; + } + + /** {@inheritDoc} */ + @Override public long coordinatorVersion() { + return crdVer; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("cleanupVer", cleanupVer)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeLong("cntr", cntr)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeLong("crdVer", crdVer)) + return false; + + writer.incrementState(); + + case 3: + if (!writer.writeLong("futId", futId)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeInt("opCntr", opCntr)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeLongArray("txs", txs)) + 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: + cleanupVer = reader.readLong("cleanupVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + cntr = reader.readLong("cntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + crdVer = reader.readLong("crdVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 3: + futId = reader.readLong("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + opCntr = reader.readInt("opCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + txs = reader.readLongArray("txs"); + + if (!reader.isLastRead()) + return false; + + txsCnt = txs != null ? txs.length : 0; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccSnapshotResponse.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 141; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 6; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccSnapshotResponse.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java new file mode 100644 index 0000000000000..cd30eb85b88af --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccTxSnapshotRequest.java @@ -0,0 +1,128 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +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; + +/** + * Request to get MVCC snapshot for a new transaction. + */ +public class MvccTxSnapshotRequest implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long futId; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public MvccTxSnapshotRequest() { + // No-op. + } + + /** + * @param futId Future ID. + */ + public MvccTxSnapshotRequest(long futId) { + this.futId = futId; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return true; + } + + /** + * @return Future ID. + */ + public long futureId() { + return futId; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("futId", futId)) + 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: + futId = reader.readLong("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(MvccTxSnapshotRequest.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 136; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccTxSnapshotRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccWaitTxsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccWaitTxsRequest.java new file mode 100644 index 0000000000000..ae57507446d3f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/msg/MvccWaitTxsRequest.java @@ -0,0 +1,159 @@ +/* + * 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.mvcc.msg; + +import java.nio.ByteBuffer; + +import org.apache.ignite.internal.util.GridLongList; +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; + +/** + * + */ +public class MvccWaitTxsRequest implements MvccMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long futId; + + /** */ + private GridLongList txs; + + /** + * + */ + public MvccWaitTxsRequest() { + // No-op. + } + + /** + * @param futId Future ID. + * @param txs Transactions to wait for. + */ + public MvccWaitTxsRequest(long futId, GridLongList txs) { + assert txs != null && txs.size() > 0 : txs; + + this.futId = futId; + this.txs = txs; + } + + /** + * @return Future ID. + */ + public long futureId() { + return futId; + } + + /** + * @return Transactions to wait for. + */ + public GridLongList transactions() { + return txs; + } + + /** {@inheritDoc} */ + @Override public boolean waitForCoordinatorInit() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean processedFromNioThread() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("futId", futId)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeMessage("txs", txs)) + 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: + futId = reader.readLong("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + txs = reader.readMessage("txs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(MvccWaitTxsRequest.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 142; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 2; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccWaitTxsRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxKey.java new file mode 100644 index 0000000000000..92aff7b650748 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxKey.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.processors.cache.mvcc.txlog; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +public class TxKey { + /** */ + private final long major; + + /** */ + private final long minor; + + /** + * @param major Major version. + * @param minor Minor version + */ + public TxKey(long major, long minor) { + this.major = major; + this.minor = minor; + } + + /** + * @return Major version. + */ + public long major() { + return major; + } + + /** + * @return Minor version. + */ + public long minor() { + return minor; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) return true; + if (o == null || o.getClass() != TxKey.class) return false; + + TxKey txKey = (TxKey) o; + + return major == txKey.major && minor == txKey.minor; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int result = (int) (major ^ (major >>> 32)); + result = 31 * result + (int) (minor ^ (minor >>> 32)); + return result; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(TxKey.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLog.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLog.java new file mode 100644 index 0000000000000..905bfc4000d13 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLog.java @@ -0,0 +1,584 @@ +/* + * 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.mvcc.txlog; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRecord; +import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; +import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseListImpl; +import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; +import org.apache.ignite.internal.util.IgniteTree; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX; +import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; + +/** + * + */ +public class TxLog implements DbCheckpointListener { + /** */ + public static final String TX_LOG_CACHE_NAME = "TxLog"; + + /** */ + public static final int TX_LOG_CACHE_ID = CU.cacheId(TX_LOG_CACHE_NAME); + + /** */ + private static final TxKey LOWEST = new TxKey(0, 0); + + /** */ + private final IgniteCacheDatabaseSharedManager mgr; + + /** */ + private ReuseListImpl reuseList; + + /** */ + private TxLogTree tree; + + /** */ + private ConcurrentMap keyMap = new ConcurrentHashMap<>(); + + /** + * + * @param ctx Kernal context. + * @param mgr Database shared manager. + */ + public TxLog(GridKernalContext ctx, IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + this.mgr = mgr; + + init(ctx); + } + + /** + * + * @param ctx Kernal context. + * @throws IgniteCheckedException If failed. + */ + private void init(GridKernalContext ctx) throws IgniteCheckedException { + if (CU.isPersistenceEnabled(ctx.config())) { + mgr.checkpointReadLock(); + + try { + IgniteWriteAheadLogManager wal = ctx.cache().context().wal(); + PageMemoryEx pageMemory = (PageMemoryEx)mgr.dataRegion(TX_LOG_CACHE_NAME).pageMemory(); + + long partMetaId = pageMemory.partitionMetaPageId(TX_LOG_CACHE_ID, 0); + long partMetaPage = pageMemory.acquirePage(TX_LOG_CACHE_ID, partMetaId); + + long treeRoot, reuseListRoot; + + boolean isNew = false; + + try { + long pageAddr = pageMemory.writeLock(TX_LOG_CACHE_ID, partMetaId, partMetaPage); + + try { + if (PageIO.getType(pageAddr) != PageIO.T_PART_META) { + // Initialize new page. + PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.latest(); + + io.initNewPage(pageAddr, partMetaId, pageMemory.pageSize()); + + treeRoot = pageMemory.allocatePage(TX_LOG_CACHE_ID, 0, PageMemory.FLAG_DATA); + reuseListRoot = pageMemory.allocatePage(TX_LOG_CACHE_ID, 0, PageMemory.FLAG_DATA); + + assert PageIdUtils.flag(treeRoot) == PageMemory.FLAG_DATA; + assert PageIdUtils.flag(reuseListRoot) == PageMemory.FLAG_DATA; + + io.setTreeRoot(pageAddr, treeRoot); + io.setReuseListRoot(pageAddr, reuseListRoot); + + if (PageHandler.isWalDeltaRecordNeeded(pageMemory, TX_LOG_CACHE_ID, partMetaId, partMetaPage, wal, null)) + wal.log(new MetaPageInitRecord( + TX_LOG_CACHE_ID, + partMetaId, + io.getType(), + io.getVersion(), + treeRoot, + reuseListRoot + )); + + isNew = true; + } + else { + PagePartitionMetaIO io = PageIO.getPageIO(pageAddr); + + treeRoot = io.getTreeRoot(pageAddr); + reuseListRoot = io.getReuseListRoot(pageAddr); + + assert PageIdUtils.flag(treeRoot) == PageMemory.FLAG_DATA : + U.hexLong(treeRoot) + ", part=" + 0 + ", TX_LOG_CACHE_ID=" + TX_LOG_CACHE_ID; + assert PageIdUtils.flag(reuseListRoot) == PageMemory.FLAG_DATA : + U.hexLong(reuseListRoot) + ", part=" + 0 + ", TX_LOG_CACHE_ID=" + TX_LOG_CACHE_ID; + } + } + finally { + pageMemory.writeUnlock(TX_LOG_CACHE_ID, partMetaId, partMetaPage, null, isNew); + } + } + finally { + pageMemory.releasePage(TX_LOG_CACHE_ID, partMetaId, partMetaPage); + } + + reuseList = new ReuseListImpl( + TX_LOG_CACHE_ID, + TX_LOG_CACHE_NAME, + pageMemory, + wal, + reuseListRoot, + isNew); + + tree = new TxLogTree(pageMemory, wal, treeRoot, reuseList, ctx.failure(), isNew); + + ((GridCacheDatabaseSharedManager)mgr).addCheckpointListener(this); + } + finally { + mgr.checkpointReadUnlock(); + } + } + else { + PageMemory pageMemory = mgr.dataRegion(TX_LOG_CACHE_NAME).pageMemory(); + ReuseList reuseList1 = mgr.reuseList(TX_LOG_CACHE_NAME); + + long treeRoot; + + if ((treeRoot = reuseList1.takeRecycledPage()) == 0L) + treeRoot = pageMemory.allocatePage(TX_LOG_CACHE_ID, INDEX_PARTITION, FLAG_IDX); + + tree = new TxLogTree(pageMemory, null, treeRoot, reuseList1, ctx.failure(), true); + } + } + + /** {@inheritDoc} */ + @Override public void onCheckpointBegin(Context ctx) throws IgniteCheckedException { + reuseList.saveMetadata(); + } + + /** + * + * @param major Major version. + * @param minor Minor version. + * @return Transaction state for given version. + * @throws IgniteCheckedException If failed. + */ + public byte get(long major, long minor) throws IgniteCheckedException { + return get(new TxKey(major, minor)); + } + + /** + * + * @param key Transaction key. + * @return Transaction state for given version. + * @throws IgniteCheckedException If failed. + */ + public byte get(TxKey key) throws IgniteCheckedException { + TxRow row = tree.findOne(key); + + return row == null ? TxState.NA : row.state(); + } + + /** + * + * @param key TxKey. + * @param state Transaction state for given version. + * @param primary Flag if this is a primary node. + * @throws IgniteCheckedException If failed. + */ + public void put(TxKey key, byte state, boolean primary) throws IgniteCheckedException { + Sync sync = syncObject(key); + + try { + mgr.checkpointReadLock(); + + try { + synchronized (sync) { + tree.invoke(key, null, new TxLogUpdateClosure(key.major(), key.minor(), state, primary)); + } + } + finally { + mgr.checkpointReadUnlock(); + } + } finally { + evict(key, sync); + } + } + + /** + * Removes all records less or equals to the given version. + * + * @param major Major version. + * @param minor Minor version. + * @throws IgniteCheckedException If failed. + */ + public void removeUntil(long major, long minor) throws IgniteCheckedException { + TraversingClosure clo = new TraversingClosure(major, minor); + + tree.iterate(LOWEST, clo, clo); + + if (clo.rows != null) { + for (TxKey row : clo.rows) { + remove(row); + } + } + } + + /** */ + private void remove(TxKey key) throws IgniteCheckedException { + Sync sync = syncObject(key); + + try { + mgr.checkpointReadLock(); + + try { + synchronized (sync) { + tree.removex(key); + } + } + finally { + mgr.checkpointReadUnlock(); + } + } finally { + evict(key, sync); + } + } + + /** */ + private Sync syncObject(TxKey key) { + Sync sync = keyMap.get(key); + + while (true) { + if (sync == null) { + Sync old = keyMap.putIfAbsent(key, sync = new Sync()); + + if (old == null) + return sync; + else + sync = old; + } + else { + int cntr = sync.counter; + + while (cntr > 0) { + if (sync.casCounter(cntr, cntr + 1)) + return sync; + + cntr = sync.counter; + } + + sync = keyMap.get(key); + } + } + } + + /** */ + private void evict(TxKey key, Sync sync) { + assert sync != null; + + int cntr = sync.counter; + + while (true) { + assert cntr > 0; + + if (!sync.casCounter(cntr, cntr - 1)) { + cntr = sync.counter; + + continue; + } + + if (cntr == 1) { + boolean removed = keyMap.remove(key, sync); + + assert removed; + } + + break; + } + } + + /** + * + */ + private static class TraversingClosure extends TxKey implements BPlusTree.TreeRowClosure { + /** */ + private List rows; + + /** + * + * @param major Major version. + * @param minor Minor version. + */ + TraversingClosure(long major, long minor) { + super(major, minor); + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, + int idx) throws IgniteCheckedException { + + if (rows == null) + rows = new ArrayList<>(); + + TxLogIO logIO = (TxLogIO)io; + int offset = io.offset(idx); + + rows.add(new TxKey(logIO.getMajor(pageAddr, offset), logIO.getMinor(pageAddr, offset))); + + return true; + } + } + + /** */ + private static class Sync { + /** */ + private static final AtomicIntegerFieldUpdater UPD = AtomicIntegerFieldUpdater.newUpdater(Sync.class, "counter"); + + /** */ + volatile int counter = 1; + + /** */ + boolean casCounter(int old, int upd) { + return UPD.compareAndSet(this, old, upd); + } + } + + /** + * TxLog update closure. + */ + private static final class TxLogUpdateClosure implements IgniteTree.InvokeClosure { + /** */ + private final long major; + + /** */ + private final long minor; + + /** */ + private final byte newState; + + /** */ + private final boolean primary; + + /** */ + private IgniteTree.OperationType treeOp; + + /** + * + * @param major Coordinator version. + * @param minor Counter. + * @param newState New Tx newState. + * @param primary Flag if this is primary node. + */ + TxLogUpdateClosure(long major, long minor, byte newState, boolean primary) { + assert major > MVCC_CRD_COUNTER_NA && minor > MVCC_COUNTER_NA && newState != TxState.NA; + this.major = major; + this.minor = minor; + this.newState = newState; + this.primary = primary; + } + + /** {@inheritDoc} */ + @Override public void call(@Nullable TxRow row) { + if (row == null) { + valid(); + + return; + } + + byte currState = row.state(); + + switch (currState) { + case TxState.NA: + checkNa(currState); + + break; + + case TxState.PREPARED: + checkPrepared(currState); + + break; + + case TxState.COMMITTED: + checkCommitted(currState); + + break; + + case TxState.ABORTED: + checkAborted(currState); + + break; + + default: + throw new IllegalStateException("Unknown tx state: " + currState); + } + } + + /** {@inheritDoc} */ + @Override public TxRow newRow() { + return treeOp == IgniteTree.OperationType.PUT ? new TxRow(major, minor, newState) : null; + } + + /** {@inheritDoc} */ + @Override public IgniteTree.OperationType operationType() { + return treeOp; + } + + /** + * Checks update possibility for {@code TxState.NA} tx status. + * + * @param currState Current tx state. + */ + private void checkNa(byte currState) { + switch (newState) { + case TxState.ABORTED: + case TxState.PREPARED: + valid(); + + break; + + case TxState.COMMITTED: + invalid(currState); // TODO IGNITE-8445 + + break; + + default: + invalid(currState); + } + } + + /** + * Checks update possibility for {@code TxState.PREPARED} status. + * + * @param currState Current tx state. + */ + private void checkPrepared(byte currState) { + switch (newState) { + case TxState.ABORTED: + case TxState.COMMITTED: + valid(); + + break; + + case TxState.PREPARED: + ignore(); + + break; + + default: + invalid(currState); + } + } + + /** + * Checks update possibility for {@code TxState.COMMITTED} status. + * + * @param currState Current tx state. + */ + private void checkCommitted(byte currState) { + switch (newState) { + case TxState.COMMITTED: + ignore(); + + break; + + case TxState.PREPARED: + if (primary) + ignore(); // In case when remote tx has updated the current state before. + else + invalid(currState); + + break; + + default: + invalid(currState); + } + } + + /** + * Checks update possibility for {@code TxState.ABORTED} status. + * + * @param currState Current tx state. + */ + private void checkAborted(byte currState) { + switch (newState) { + case TxState.ABORTED: + ignore(); + + break; + + case TxState.PREPARED: + if (primary) + ignore(); // In case when remote tx has updated the current state before. + else + invalid(currState); + + break; + + default: + invalid(currState); + } + } + + /** + * Action for valid tx status update. + */ + private void valid() { + assert treeOp == null; + + treeOp = IgniteTree.OperationType.PUT; + } + + /** + * Action for invalid tx status update. + */ + private void invalid(byte currState) { + assert treeOp == null; + + throw new IllegalStateException("Unexpected new transaction state. [currState=" + + currState + ", newState=" + newState + ", cntr=" + minor +']'); + } + + /** + * Action for ignoring tx status update. + */ + private void ignore() { + assert treeOp == null; + + treeOp = IgniteTree.OperationType.NOOP; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogIO.java new file mode 100644 index 0000000000000..e952b433ac89d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogIO.java @@ -0,0 +1,73 @@ +/* + * 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.mvcc.txlog; + +/** + * + */ +public interface TxLogIO { + /** + * @param pageAddr Page address. + * @param off Item offset. + * @param row Row to compare with. + * @return Comparision result. + */ + int compare(long pageAddr, int off, TxKey row); + + /** + * @param pageAddr Page address. + * @param off Item offset. + * @return Major version + */ + long getMajor(long pageAddr, int off); + + /** + * @param pageAddr Page address. + * @param off Item offset. + * @param major Major version + */ + void setMajor(long pageAddr, int off, long major); + + /** + * @param pageAddr Page address. + * @param off Item offset. + * @return Minor version. + */ + long getMinor(long pageAddr, int off); + + /** + * @param pageAddr Page address. + * @param off Item offset. + * @param minor Minor version. + */ + void setMinor(long pageAddr, int off, long minor); + + /** + * @param pageAddr Page address. + * @param off Item offset. + * @return Transaction state. + */ + byte getState(long pageAddr, int off); + + /** + * @param pageAddr Page address. + * @param off Item offset. + * @param state Transaction state. + */ + void setState(long pageAddr, int off, byte state); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogInnerIO.java new file mode 100644 index 0000000000000..95c10ce0fa7d4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogInnerIO.java @@ -0,0 +1,105 @@ +/* + * 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.mvcc.txlog; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; + +/** */ +public class TxLogInnerIO extends BPlusInnerIO implements TxLogIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>(new TxLogInnerIO(1)); + + /** + * @param ver Page format version. + */ + protected TxLogInnerIO(int ver) { + super(T_TX_LOG_INNER, ver, true, 8 + 8 + 1); + } + + /** {@inheritDoc} */ + @Override public void storeByOffset(long pageAddr, int off, TxKey row) { + TxRow row0 = (TxRow)row; + + setMajor(pageAddr, off, row0.major()); + setMinor(pageAddr, off, row0.minor()); + setState(pageAddr, off, row0.state()); + } + + /** {@inheritDoc} */ + @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { + TxLogIO srcIo0 = (TxLogIO)srcIo; + + int srcOff = srcIo.offset(srcIdx); + int dstOff = offset(dstIdx); + + setMajor(dstPageAddr, dstOff, srcIo0.getMajor(srcPageAddr, srcOff)); + setMinor(dstPageAddr, dstOff, srcIo0.getMinor(srcPageAddr, srcOff)); + setState(dstPageAddr, dstOff, srcIo0.getState(srcPageAddr, srcOff)); + } + + /** {@inheritDoc} */ + @Override public TxKey getLookupRow(BPlusTree tree, long pageAddr, int idx) { + int off = offset(idx); + + return new TxRow( + getMajor(pageAddr, off), + getMinor(pageAddr, off), + getState(pageAddr, off)); + } + + /** {@inheritDoc} */ + @Override public int compare(long pageAddr, int off, TxKey row) { + int cmp = Long.compare(PageUtils.getLong(pageAddr, off), row.major()); + + return cmp != 0 ? cmp : Long.compare(getMinor(pageAddr, off), row.minor()); + } + + /** {@inheritDoc} */ + @Override public long getMajor(long pageAddr, int off) { + return PageUtils.getLong(pageAddr, off); + } + + /** {@inheritDoc} */ + @Override public void setMajor(long pageAddr, int off, long major) { + PageUtils.putLong(pageAddr, off, major); + } + + /** {@inheritDoc} */ + @Override public long getMinor(long pageAddr, int off) { + return PageUtils.getLong(pageAddr, off + 8); + } + + /** {@inheritDoc} */ + @Override public void setMinor(long pageAddr, int off, long minor) { + PageUtils.putLong(pageAddr, off + 8, minor); + } + + /** {@inheritDoc} */ + @Override public byte getState(long pageAddr, int off) { + return PageUtils.getByte(pageAddr, off + 16); + } + + /** {@inheritDoc} */ + @Override public void setState(long pageAddr, int off, byte state) { + PageUtils.putByte(pageAddr, off + 16, state); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogLeafIO.java new file mode 100644 index 0000000000000..e037fbe9cb2ec --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogLeafIO.java @@ -0,0 +1,105 @@ +/* + * 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.mvcc.txlog; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; + +/** */ +public class TxLogLeafIO extends BPlusLeafIO implements TxLogIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>(new TxLogLeafIO(1)); + + /** + * @param ver Page format version. + */ + protected TxLogLeafIO(int ver) { + super(T_TX_LOG_LEAF, ver, 17); + } + + /** {@inheritDoc} */ + @Override public void storeByOffset(long pageAddr, int off, TxKey row) { + TxRow row0 = (TxRow)row; + + setMajor(pageAddr, off, row0.major()); + setMinor(pageAddr, off, row0.minor()); + setState(pageAddr, off, row0.state()); + } + + /** {@inheritDoc} */ + @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { + TxLogIO srcIo0 = (TxLogIO)srcIo; + + int srcOff = srcIo.offset(srcIdx); + int dstOff = offset(dstIdx); + + setMajor(dstPageAddr, dstOff, srcIo0.getMajor(srcPageAddr, srcOff)); + setMinor(dstPageAddr, dstOff, srcIo0.getMinor(srcPageAddr, srcOff)); + setState(dstPageAddr, dstOff, srcIo0.getState(srcPageAddr, srcOff)); + } + + /** {@inheritDoc} */ + @Override public TxKey getLookupRow(BPlusTree tree, long pageAddr, int idx) { + int off = offset(idx); + + return new TxRow( + getMajor(pageAddr, off), + getMinor(pageAddr, off), + getState(pageAddr, off)); + } + + /** {@inheritDoc} */ + @Override public int compare(long pageAddr, int off, TxKey row) { + int cmp = Long.compare(getMajor(pageAddr, off), row.major()); + + return cmp != 0 ? cmp : Long.compare(getMinor(pageAddr, off), row.minor()); + } + + /** {@inheritDoc} */ + @Override public long getMajor(long pageAddr, int off) { + return PageUtils.getLong(pageAddr, off); + } + + /** {@inheritDoc} */ + @Override public void setMajor(long pageAddr, int off, long major) { + PageUtils.putLong(pageAddr, off, major); + } + + /** {@inheritDoc} */ + @Override public long getMinor(long pageAddr, int off) { + return PageUtils.getLong(pageAddr, off + 8); + } + + /** {@inheritDoc} */ + @Override public void setMinor(long pageAddr, int off, long minor) { + PageUtils.putLong(pageAddr, off + 8, minor); + } + + /** {@inheritDoc} */ + @Override public byte getState(long pageAddr, int off) { + return PageUtils.getByte(pageAddr, off + 16); + } + + /** {@inheritDoc} */ + @Override public void setState(long pageAddr, int off, byte state) { + PageUtils.putByte(pageAddr, off + 16, state); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogTree.java new file mode 100644 index 0000000000000..60fbc84dd5f54 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxLogTree.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc.txlog; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; +import org.apache.ignite.internal.processors.failure.FailureProcessor; + +/** + * + */ +public class TxLogTree extends BPlusTree { + /** + * @param pageMem Page memory. + * @param wal Write ahead log manager + * @param metaPageId Tree metapage id. + * @param reuseList Reuse list + * @param failureProcessor Failure processor. + * @param initNew {@code True} if new tree should be created. + * @throws IgniteCheckedException If fails. + */ + public TxLogTree(PageMemory pageMem, + IgniteWriteAheadLogManager wal, long metaPageId, + ReuseList reuseList, FailureProcessor failureProcessor, + boolean initNew) throws IgniteCheckedException { + super(TxLog.TX_LOG_CACHE_NAME, TxLog.TX_LOG_CACHE_ID, pageMem, wal, new AtomicLong(), metaPageId, + reuseList, TxLogInnerIO.VERSIONS, TxLogLeafIO.VERSIONS, failureProcessor); + + initTree(initNew); + } + + /** {@inheritDoc} */ + @Override protected int compare(BPlusIO io, long pageAddr, int idx, TxKey row) { + return ((TxLogIO)io).compare(pageAddr, io.offset(idx), row); + } + + /** {@inheritDoc} */ + @Override public TxRow getRow(BPlusIO io, long pageAddr, + int idx, Object ignored) throws IgniteCheckedException { + return (TxRow) io.getLookupRow(this, pageAddr, idx); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxRow.java new file mode 100644 index 0000000000000..0d161c844f90a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxRow.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc.txlog; + +/** + * + */ +public class TxRow extends TxKey { + /** */ + private byte state; + + /** + * @param major Major version. + * @param minor Minor version. + * @param state Transaction state. + */ + TxRow(long major, long minor, byte state) { + super(major, minor); + + this.state = state; + } + + /** + * @return Transaction state. + */ + public byte state() { + return state; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxState.java new file mode 100644 index 0000000000000..65a1f251a98a0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/txlog/TxState.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.internal.processors.cache.mvcc.txlog; + +/** + * + */ +public final class TxState { + /** */ + public static final byte NA = 0x0; + /** */ + public static final byte PREPARED = 0x1; + /** */ + public static final byte ABORTED = 0x2; + /** */ + public static final byte COMMITTED = 0x3; + + /** + * Private constructor. + */ + private TxState() {} +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java index 44f0a3f3b0b95..64517acf27a2a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java @@ -19,12 +19,13 @@ import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; /** * Cache data row. */ -public interface CacheDataRow extends CacheSearchRow, Storable { +public interface CacheDataRow extends MvccUpdateVersionAware, CacheSearchRow, Storable { /** * @return Cache value. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java index 9f2e031076a48..b8245df0422a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.IncompleteCacheObject; import org.apache.ignite.internal.processors.cache.IncompleteObject; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPagePayload; @@ -42,6 +43,10 @@ import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId; import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.LINK_WITH_HEADER; /** * Cache data row adapter. @@ -60,6 +65,7 @@ public class CacheDataRowAdapter implements CacheDataRow { protected CacheObject val; /** */ + @GridToStringInclude protected long expireTime = -1; /** */ @@ -153,6 +159,8 @@ public final void initFromLink( nextLink = data.nextLink(); + int hdrLen = 0; + if (first) { if (nextLink == 0) { // Fast path for a single page row. @@ -162,12 +170,21 @@ public final void initFromLink( } first = false; + + // Assume that row header is always located entirely on the very first page. + hdrLen = readHeader(pageAddr, data.offset()); + + if (rowData == LINK_WITH_HEADER) + return; } ByteBuffer buf = pageMem.pageBuffer(pageAddr); - buf.position(data.offset()); - buf.limit(data.offset() + data.payloadSize()); + int off = data.offset() + hdrLen; + int payloadSize = data.payloadSize() - hdrLen; + + buf.position(off); + buf.limit(off + payloadSize); boolean keyOnly = rowData == RowData.KEY_ONLY; @@ -189,6 +206,18 @@ public final void initFromLink( assert isReady() : "ready"; } + /** + * Reads row header (i.e. MVCC info) which should be located on the very first page od data. + * + * @param addr Address. + * @param off Offset + * @return Number of bytes read. + */ + protected int readHeader(long addr, int off) { + // No-op. + return 0; + } + /** * @param sharedCtx Cache shared context. * @param coctx Cache object context. @@ -199,7 +228,7 @@ public final void initFromLink( * @throws IgniteCheckedException If failed. * @return Read object. */ - private IncompleteObject readFragment( + protected IncompleteObject readFragment( GridCacheSharedContext sharedCtx, CacheObjectContext coctx, ByteBuffer buf, @@ -268,7 +297,7 @@ private IncompleteObject readFragment( * @param readCacheId {@code true} If need to read cache ID. * @throws IgniteCheckedException If failed. */ - private void readFullRow( + protected void readFullRow( GridCacheSharedContext sharedCtx, CacheObjectContext coctx, long addr, @@ -277,6 +306,11 @@ private void readFullRow( throws IgniteCheckedException { int off = 0; + off += readHeader(addr, off); + + if (rowData == LINK_WITH_HEADER) + return; + if (readCacheId) { cacheId = PageUtils.getInt(addr, off); @@ -326,7 +360,7 @@ private void readFullRow( * @param buf Buffer. * @param incomplete Incomplete. */ - private IncompleteObject readIncompleteCacheId( + protected IncompleteObject readIncompleteCacheId( ByteBuffer buf, IncompleteObject incomplete ) { @@ -371,7 +405,7 @@ private IncompleteObject readIncompleteCacheId( * @return Incomplete object. * @throws IgniteCheckedException If failed. */ - private IncompleteCacheObject readIncompleteKey( + protected IncompleteCacheObject readIncompleteKey( CacheObjectContext coctx, ByteBuffer buf, IncompleteCacheObject incomplete @@ -396,7 +430,7 @@ private IncompleteCacheObject readIncompleteKey( * @return Incomplete object. * @throws IgniteCheckedException If failed. */ - private IncompleteCacheObject readIncompleteValue( + protected IncompleteCacheObject readIncompleteValue( CacheObjectContext coctx, ByteBuffer buf, IncompleteCacheObject incomplete @@ -419,7 +453,7 @@ private IncompleteCacheObject readIncompleteValue( * @param incomplete Incomplete object. * @return Incomplete object. */ - private IncompleteObject readIncompleteExpireTime( + protected IncompleteObject readIncompleteExpireTime( ByteBuffer buf, IncompleteObject incomplete ) { @@ -463,7 +497,7 @@ private IncompleteObject readIncompleteExpireTime( * @return Incomplete object. * @throws IgniteCheckedException If failed. */ - private IncompleteObject readIncompleteVersion( + protected IncompleteObject readIncompleteVersion( ByteBuffer buf, IncompleteObject incomplete ) throws IgniteCheckedException { @@ -573,6 +607,60 @@ public boolean isReady() { throw new UnsupportedOperationException(); } + /** {@inheritDoc} */ + @Override public int size() throws IgniteCheckedException { + int len = key().valueBytesLength(null); + + len += value().valueBytesLength(null) + CacheVersionIO.size(version(), false) + 8; + + return len + (cacheId() != 0 ? 4 : 0); + } + + /** {@inheritDoc} */ + @Override public int headerSize() { + return 0; + } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return MVCC_CRD_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return MVCC_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return MVCC_OP_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public byte mvccTxState() { + return TxState.NA; + } + + /** {@inheritDoc} */ + @Override public long newMvccCoordinatorVersion() { + return MVCC_CRD_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public long newMvccCounter() { + return MVCC_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public int newMvccOperationCounter() { + return MVCC_OP_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public byte newMvccTxState() { + return TxState.NA; + } + /** * */ @@ -584,7 +672,13 @@ public enum RowData { KEY_ONLY, /** */ - NO_KEY + NO_KEY, + + /** */ + LINK_ONLY, + + /** */ + LINK_WITH_HEADER } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheSearchRow.java index 1637eb05c4719..c3cfb83d57d0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheSearchRow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheSearchRow.java @@ -18,11 +18,12 @@ package org.apache.ignite.internal.processors.cache.persistence; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; /** * */ -public interface CacheSearchRow { +public interface CacheSearchRow extends MvccVersionAware { /** * @return Cache key. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java new file mode 100644 index 0000000000000..f96cdd91a7e8f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence; + +import org.apache.ignite.IgniteCheckedException; + +/** + * + */ +public interface DatabaseLifecycleListener { + + /** + * @param mgr Database shared manager. + * + */ + void onInitDataRegions(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + + /** + * @param mgr Page store manager. + * + */ + void beforeMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + + /** + * @param mgr Database shared manager. + * + */ + void afterMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + + /** + * @param mgr Database shared manager. + */ + void afterInitialise(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + + /** + * @param mgr Database shared manager. + */ + void beforeStop(IgniteCacheDatabaseSharedManager mgr); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index bd8a62f8fcc62..7d91645b57e64 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -111,6 +111,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntry; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntryType; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointHistory; @@ -409,8 +410,8 @@ public IgniteInternalFuture enableCheckpoints(boolean enable) { } /** {@inheritDoc} */ - @Override protected void initDataRegions(DataStorageConfiguration memCfg) throws IgniteCheckedException { - super.initDataRegions(memCfg); + @Override protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCheckedException { + super.initDataRegions0(memCfg); addDataRegion( memCfg, @@ -812,6 +813,10 @@ private void unRegistrateMetricsMBean() { checkpointReadLock(); try { + for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) { + lsnr.beforeMemoryRestore(this); + } + if (!F.isEmpty(cachesToStart)) { for (DynamicCacheDescriptor desc : cachesToStart) { if (CU.affinityNode(cctx.localNode(), desc.cacheConfiguration().getNodeFilter())) @@ -851,6 +856,11 @@ private void unRegistrateMetricsMBean() { metaStorage.init(this); notifyMetastorageReadyForReadWrite(); + + for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) { + lsnr.afterMemoryRestore(this); + } + } catch (IgniteCheckedException e) { if (X.hasCause(e, StorageException.class, IOException.class)) @@ -1351,7 +1361,7 @@ private void prepareIndexRebuildFuture(int cacheId) { if (!cctx.pageStore().hasIndexStore(cacheCtx.groupId()) && cacheCtx.affinityNode()) { IgniteInternalFuture rebuildFut = cctx.kernalContext().query() - .rebuildIndexesFromHash(Collections.singletonList(cacheCtx.cacheId())); + .rebuildIndexesFromHash(Collections.singleton(cacheCtx.cacheId())); assert usrFut != null : "Missing user future for cache: " + cacheCtx.name(); @@ -2088,6 +2098,10 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC * @throws IgniteCheckedException if no DataRegion is configured for a name obtained from cache descriptor. */ private PageMemoryEx getPageMemoryForCacheGroup(int grpId) throws IgniteCheckedException { + // TODO IGNITE-7792 add generic mapping. + if (grpId == TxLog.TX_LOG_CACHE_ID) + return (PageMemoryEx)dataRegion(TxLog.TX_LOG_CACHE_NAME).pageMemory(); + // TODO IGNITE-5075: cache descriptor can be removed. GridCacheSharedContext sharedCtx = context(); @@ -2391,7 +2405,7 @@ private long restorePartitionStates( } } else - updateState(part, (int)io.getPartitionState(pageAddr)); + changed = updateState(part, (int)io.getPartitionState(pageAddr)); } finally { pageMem.writeUnlock(grpId, partMetaId, partMetaPage, null, changed); @@ -3852,20 +3866,21 @@ private List writePages(Collection writePageIds) throws PageMemoryEx pageMem; - if (grpId != MetaStorage.METASTORAGE_CACHE_ID) { + // TODO IGNITE-7792 add generic mapping. + if (grpId == MetaStorage.METASTORAGE_CACHE_ID) + pageMem = (PageMemoryEx)metaStorage.pageMemory(); + else if (grpId == TxLog.TX_LOG_CACHE_ID) + pageMem = (PageMemoryEx)dataRegion(TxLog.TX_LOG_CACHE_NAME).pageMemory(); + else { CacheGroupContext grp = context().cache().cacheGroup(grpId); - if (grp == null) - continue; + DataRegion region = grp != null ?grp .dataRegion() : null; - if (!grp.dataRegion().config().isPersistenceEnabled()) + if (region == null || !region.config().isPersistenceEnabled()) continue; - pageMem = (PageMemoryEx)grp.dataRegion().pageMemory(); + pageMem = (PageMemoryEx)region.pageMemory(); } - else - pageMem = (PageMemoryEx)metaStorage.pageMemory(); - Integer tag = pageMem.getForCheckpoint( fullId, tmpWriteBuf, persStoreMetrics.metricsEnabled() ? tracker : null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index 826f6191ff760..4c45352ce5585 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -17,9 +17,11 @@ package org.apache.ignite.internal.processors.cache.persistence; +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.NoSuchElementException; import java.util.Set; @@ -57,6 +59,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteHistoricalIterator; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl; import org.apache.ignite.internal.processors.cache.persistence.migration.UpgradePendingTreeToPerPartitionTask; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; @@ -76,8 +80,11 @@ import org.apache.ignite.internal.processors.cache.tree.CacheDataTree; import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; import org.apache.ignite.internal.processors.cache.tree.PendingRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -735,11 +742,14 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override @Nullable protected WALHistoricalIterator historicalIterator( + @Override @Nullable protected IgniteHistoricalIterator historicalIterator( CachePartitionPartialCountersMap partCntrs, Set missing) throws IgniteCheckedException { if (partCntrs == null || partCntrs.isEmpty()) return null; + if (grp.mvccEnabled()) // TODO IGNITE-7384 + return super.historicalIterator(partCntrs, missing); + GridCacheDatabaseSharedManager database = (GridCacheDatabaseSharedManager)grp.shared().database(); FileWALPointer minPtr = null; @@ -1092,6 +1102,16 @@ private DataEntryRow(DataEntry entry) { return entry.partitionId(); } + /** {@inheritDoc} */ + @Override public int size() throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public int headerSize() { + throw new UnsupportedOperationException(); + } + /** {@inheritDoc} */ @Override public long link() { return 0; @@ -1111,6 +1131,46 @@ private DataEntryRow(DataEntry entry) { @Override public int cacheId() { return entry.cacheId(); } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return 0; // TODO IGNITE-7384 + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return 0; // TODO IGNITE-7384 + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return 0; // TODO IGNITE-7384 + } + + /** {@inheritDoc} */ + @Override public long newMvccCoordinatorVersion() { + return 0; // TODO IGNITE-7384 + } + + /** {@inheritDoc} */ + @Override public long newMvccCounter() { + return 0; // TODO IGNITE-7384 + } + + /** {@inheritDoc} */ + @Override public int newMvccOperationCounter() { + return 0; // TODO IGNITE-7384 + } + + /** {@inheritDoc} */ + @Override public byte mvccTxState() { + return 0; // TODO IGNITE-7384 + } + + /** {@inheritDoc} */ + @Override public byte newMvccTxState() { + return 0; // TODO IGNITE-7384 + } } /** @@ -1504,6 +1564,30 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { } } + /** {@inheritDoc} */ + @Override public long nextMvccUpdateCounter() { + try { + CacheDataStore delegate0 = init0(true); + + return delegate0 == null ? 0 : delegate0.nextMvccUpdateCounter(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override public long mvccUpdateCounter() { + try { + CacheDataStore delegate0 = init0(true); + + return delegate0 == null ? 0 : delegate0.mvccUpdateCounter(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + /** {@inheritDoc} */ @Override public void init(long size, long updCntr, @Nullable Map cacheSizes) { throw new IllegalStateException("Should be never called."); @@ -1588,6 +1672,126 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { delegate.update(cctx, key, val, ver, expireTime, oldRow); } + /** {@inheritDoc} */ + @Override public boolean mvccInitialValue( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer) + throws IgniteCheckedException + { + CacheDataStore delegate = init0(false); + + return delegate.mvccInitialValue(cctx, key, val, ver, expireTime, mvccVer, newMvccVer); + } + + /** {@inheritDoc} */ + @Override public boolean mvccInitialValueIfAbsent( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte txState, + byte newTxState) + throws IgniteCheckedException + { + CacheDataStore delegate = init0(false); + + return delegate.mvccInitialValueIfAbsent(cctx, key, val, ver, expireTime, mvccVer, newMvccVer, + txState, newTxState); + } + + /** {@inheritDoc} */ + @Override public boolean mvccUpdateRowWithPreloadInfo( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState) throws IgniteCheckedException { + + CacheDataStore delegate = init0(false); + + return delegate.mvccUpdateRowWithPreloadInfo(cctx, + key, + val, + ver, + expireTime, + mvccVer, + newMvccVer, + mvccTxState, + newMvccTxState); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccUpdate( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccVer, + boolean primary, + boolean needHistory, + boolean noCreate) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + return delegate.mvccUpdate( + cctx, key, val, ver, expireTime, mvccVer, primary, needHistory, noCreate); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccRemove( + GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot mvccVer, + boolean primary, + boolean needHistory) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + return delegate.mvccRemove(cctx, key, mvccVer, primary, needHistory); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccLock( + GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + return delegate.mvccLock(cctx, key, mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public GridLongList mvccUpdateNative(GridCacheContext cctx, boolean primary, KeyCacheObject key, CacheObject val, GridCacheVersion ver, long expireTime, MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + return delegate.mvccUpdateNative(cctx, primary, key, val, ver, expireTime, mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public GridLongList mvccRemoveNative(GridCacheContext cctx, boolean primary, KeyCacheObject key, MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + return delegate.mvccRemoveNative(cctx, primary, key, mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public void mvccRemoveAll(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + delegate.mvccRemoveAll(cctx, key); + } + /** {@inheritDoc} */ @Override public CacheDataRow createRow( GridCacheContext cctx, @@ -1603,6 +1807,21 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { return delegate.createRow(cctx, key, val, ver, expireTime, oldRow); } + /** {@inheritDoc} */ + @Override public int cleanup(GridCacheContext cctx, + @Nullable List cleanupRows) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + return delegate.cleanup(cctx, cleanupRows); + } + + /** {@inheritDoc} */ + @Override public void updateTxState(GridCacheContext cctx, CacheSearchRow row) throws IgniteCheckedException { + CacheDataStore delegate = init0(false); + + delegate.updateTxState(cctx, row); + } + /** {@inheritDoc} */ @Override public void invoke(GridCacheContext cctx, KeyCacheObject key, OffheapInvokeClosure c) throws IgniteCheckedException { @@ -1633,6 +1852,40 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { return null; } + /** {@inheritDoc} */ + @Override public CacheDataRow mvccFind(GridCacheContext cctx, KeyCacheObject key, MvccSnapshot snapshot) + throws IgniteCheckedException { + CacheDataStore delegate = init0(true); + + if (delegate != null) + return delegate.mvccFind(cctx, key, snapshot); + + return null; + } + + /** {@inheritDoc} */ + @Override public List> mvccFindAllVersions(GridCacheContext cctx, KeyCacheObject key) + throws IgniteCheckedException { + CacheDataStore delegate = init0(true); + + if (delegate != null) + return delegate.mvccFindAllVersions(cctx, key); + + return Collections.emptyList(); + } + + /** {@inheritDoc} */ + @Override public GridCursor mvccAllVersionsCursor(GridCacheContext cctx, + KeyCacheObject key, Object x) throws IgniteCheckedException { + CacheDataStore delegate = init0(true); + + if (delegate != null) + return delegate.mvccAllVersionsCursor(cctx, key, x); + + return EMPTY_CURSOR; + } + + /** {@inheritDoc} */ @Override public GridCursor cursor() throws IgniteCheckedException { CacheDataStore delegate = init0(true); @@ -1643,6 +1896,27 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { return EMPTY_CURSOR; } + /** {@inheritDoc} */ + @Override public GridCursor cursor(Object x) throws IgniteCheckedException { + CacheDataStore delegate = init0(true); + + if (delegate != null) + return delegate.cursor(x); + + return EMPTY_CURSOR; + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(MvccSnapshot mvccSnapshot) + throws IgniteCheckedException { + CacheDataStore delegate = init0(true); + + if (delegate != null) + return delegate.cursor(mvccSnapshot); + + return EMPTY_CURSOR; + } + /** {@inheritDoc} */ @Override public GridCursor cursor( int cacheId, @@ -1670,6 +1944,21 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { return EMPTY_CURSOR; } + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId, + KeyCacheObject lower, + KeyCacheObject upper, + Object x, + MvccSnapshot mvccSnapshot) + throws IgniteCheckedException { + CacheDataStore delegate = init0(true); + + if (delegate != null) + return delegate.cursor(cacheId, lower, upper, x, mvccSnapshot); + + return EMPTY_CURSOR; + } + /** {@inheritDoc} */ @Override public void destroy() throws IgniteCheckedException { // No need to destroy delegate. @@ -1685,6 +1974,17 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { return EMPTY_CURSOR; } + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + CacheDataStore delegate = init0(true); + + if (delegate != null) + return delegate.cursor(cacheId, mvccSnapshot); + + return EMPTY_CURSOR; + } + /** {@inheritDoc} */ @Override public void clear(int cacheId) throws IgniteCheckedException { CacheDataStore delegate0 = init0(true); @@ -1855,7 +2155,7 @@ private int purgeExpiredInternal(GridCacheContext cctx, /** * */ - private static final GridCursor EMPTY_CURSOR = new GridCursor() { + public static final GridCursor EMPTY_CURSOR = new GridCursor() { /** {@inheritDoc} */ @Override public boolean next() { return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java index f17e52714d5d2..3ade2657c4995 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java @@ -223,6 +223,16 @@ protected void initDataRegions(DataStorageConfiguration memCfg) throws IgniteChe if (dataRegionsInitialized) return; + initDataRegions0(memCfg); + + dataRegionsInitialized = true; + } + + /** + * @param memCfg Database config. + * @throws IgniteCheckedException If failed to initialize swap path. + */ + protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCheckedException { DataRegionConfiguration[] dataRegionCfgs = memCfg.getDataRegionConfigurations(); int dataRegions = dataRegionCfgs == null ? 0 : dataRegionCfgs.length; @@ -251,8 +261,17 @@ protected void initDataRegions(DataStorageConfiguration memCfg) throws IgniteChe CU.isPersistenceEnabled(memCfg) ); + for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) { + lsnr.onInitDataRegions(this); + } + } - dataRegionsInitialized = true; + /** + * @param kctx Kernal context. + * @return Database lifecycle listeners. + */ + protected List getDatabaseListeners(GridKernalContext kctx) { + return kctx.internalSubscriptionProcessor().getDatabaseListeners(); } /** @@ -260,7 +279,7 @@ protected void initDataRegions(DataStorageConfiguration memCfg) throws IgniteChe * @param dataRegionCfg Data region config. * @throws IgniteCheckedException If failed to initialize swap path. */ - protected void addDataRegion( + public void addDataRegion( DataStorageConfiguration dataStorageCfg, DataRegionConfiguration dataRegionCfg, boolean trackable @@ -984,6 +1003,7 @@ protected DirectMemoryProvider wrapMetricsMemoryProvider( final DataRegionMetricsImpl memMetrics ) { return new DirectMemoryProvider() { + /** */ private final DirectMemoryProvider memProvider = memoryProvider0; @Override public void initialize(long[] chunkSizes) { @@ -1039,10 +1059,18 @@ protected File buildPath(String path, String consId) throws IgniteCheckedExcepti startMemoryPolicies(); initPageMemoryDataStructures(memCfg); + + for (DatabaseLifecycleListener lsnr : getDatabaseListeners(kctx)) { + lsnr.afterInitialise(this); + } } /** {@inheritDoc} */ @Override public void onDeActivate(GridKernalContext kctx) { + for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) { + lsnr.beforeStop(this); + } + if (dataRegionMap != null) { for (DataRegion memPlc : dataRegionMap.values()) { memPlc.pageMemory().stop(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java index a4266fa83416f..6248765525866 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java @@ -220,7 +220,7 @@ private MetaTree( } /** {@inheritDoc} */ - @Override protected IndexItem getRow(final BPlusIO io, final long pageAddr, + @Override public IndexItem getRow(final BPlusIO io, final long pageAddr, final int idx, Object ignore) throws IgniteCheckedException { return readRow(pageAddr, ((IndexIO)io).getOffset(pageAddr, idx)); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java index ee7c255326b2d..6900b7ee6a0e7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; +import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; /** @@ -100,6 +101,8 @@ public void addRow(CacheDataRow row) throws IgniteCheckedException { try { freeList.insertDataRow(row); + + assert row.link() != 0L; } finally { ctx.database().checkpointReadUnlock(); @@ -110,8 +113,8 @@ public void addRow(CacheDataRow row) throws IgniteCheckedException { /** * @param link Row link. * @param row New row data. - * @throws IgniteCheckedException If failed. * @return {@code True} if was able to update row. + * @throws IgniteCheckedException If failed. */ public boolean updateRow(long link, CacheDataRow row) throws IgniteCheckedException { assert !persistenceEnabled || ctx.database().checkpointLockIsHeldByThread(); @@ -122,6 +125,29 @@ public boolean updateRow(long link, CacheDataRow row) throws IgniteCheckedExcept return freeList.updateDataRow(link, row); } + /** + * Run page handler operation over the row. + * + * @param link Row link. + * @param pageHnd Page handler. + * @param arg Page handler argument. + * @throws IgniteCheckedException If failed. + */ + public void updateDataRow(long link, PageHandler pageHnd, S arg) throws IgniteCheckedException { + if (!persistenceEnabled) + freeList.updateDataRow(link, pageHnd, arg); + else { + ctx.database().checkpointReadLock(); + + try { + freeList.updateDataRow(link, pageHnd, arg); + } + finally { + ctx.database().checkpointReadUnlock(); + } + } + } + /** * @return Free list. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java index ae200dfd4f6f7..133f0a10afd06 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.cache.persistence; +import org.apache.ignite.IgniteCheckedException; + /** * Simple interface for data, store in some RowStore. */ @@ -35,4 +37,16 @@ public interface Storable { * @return Partition. */ public int partition(); + + /** + * @return Row size in page. + * @throws IgniteCheckedException If failed. + */ + public int size() throws IgniteCheckedException; + + /** + * @return Row header size in page. Header is indivisible part of row + * which is entirely available on the very first page followed by the row link. + */ + public int headerSize(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java index 933d28eb19cde..aa092419a70e6 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java @@ -278,6 +278,23 @@ public FilePageStoreManager(GridKernalContext ctx) { } } + /** {@inheritDoc} */ + @Override public void initialize(int cacheId, int partitions, String workingDir, AllocatedPageTracker tracker) + throws IgniteCheckedException { + if (!idxCacheStores.containsKey(cacheId)) { + CacheStoreHolder holder = initDir( + new File(storeWorkDir, workingDir), + cacheId, + partitions, + tracker + ); + + CacheStoreHolder old = idxCacheStores.put(cacheId, holder); + + assert old == null : "Non-null old store holder for cacheId: " + cacheId; + } + } + /** {@inheritDoc} */ @Override public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cacheData) throws IgniteCheckedException { int grpId = grpDesc.groupId(); @@ -298,9 +315,9 @@ public FilePageStoreManager(GridKernalContext ctx) { if (!idxCacheStores.containsKey(grpId)) { CacheStoreHolder holder = initDir( new File(storeWorkDir, META_STORAGE_NAME), - grpId, - 1, - AllocatedPageTracker.NO_OP); + grpId, + 1, + AllocatedPageTracker.NO_OP ); CacheStoreHolder old = idxCacheStores.put(grpId, holder); @@ -502,7 +519,7 @@ private CacheStoreHolder initForCache(CacheGroupDescriptor grpDesc, CacheConfigu * @param cacheWorkDir Work directory. * @param grpId Group ID. * @param partitions Number of partitions. - * @param allocatedTracker Metrics updater + * @param allocatedTracker Metrics updater. * @return Cache store holder. * @throws IgniteCheckedException If failed. */ @@ -1012,7 +1029,7 @@ private static class CacheStoreHolder { /** * */ - public CacheStoreHolder(FilePageStore idxStore, FilePageStore[] partStores) { + CacheStoreHolder(FilePageStore idxStore, FilePageStore[] partStores) { this.idxStore = idxStore; this.partStores = partStores; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java index bcedd8c87a0ae..4e1f783b2c287 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java @@ -100,7 +100,7 @@ private final class UpdateRowHandler extends PageHandler { throws IgniteCheckedException { AbstractDataPageIO io = (AbstractDataPageIO)iox; - int rowSize = io.getRowSize(row); + int rowSize = row.size(); boolean updated = io.updateRow(pageAddr, itemId, pageSize(), null, row, rowSize); @@ -146,7 +146,7 @@ private final class WriteRowHandler extends PageHandler { throws IgniteCheckedException { AbstractDataPageIO io = (AbstractDataPageIO)iox; - int rowSize = io.getRowSize(row); + int rowSize = row.size(); int oldFreeSpace = io.getFreeSpace(pageAddr); assert oldFreeSpace > 0 : oldFreeSpace; @@ -466,7 +466,7 @@ private long allocateDataPage(int part) throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void insertDataRow(T row) throws IgniteCheckedException { - int rowSize = ioVersions().latest().getRowSize(row); + int rowSize = row.size(); int written = 0; @@ -545,6 +545,20 @@ private long initReusedPage(long reusedPageId, int partId) throws IgniteCheckedE return updated; } + /** {@inheritDoc} */ + @Override public R updateDataRow(long link, PageHandler pageHnd, S arg) throws IgniteCheckedException { + assert link != 0; + + long pageId = PageIdUtils.pageId(link); + int itemId = PageIdUtils.itemId(link); + + R updRes = write(pageId, pageHnd, arg, itemId, null); + + assert updRes != null; // Can't fail here. + + return updRes; + } + /** {@inheritDoc} */ @Override public void removeDataRowByLink(long link) throws IgniteCheckedException { assert link != 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java index bdca21cfff870..e73124e44e793 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java @@ -20,6 +20,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.cache.persistence.Storable; +import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; /** */ @@ -38,6 +39,17 @@ public interface FreeList { */ public boolean updateDataRow(long link, T row) throws IgniteCheckedException; + /** + * @param link Row link. + * @param pageHnd Page handler. + * @param arg Handler argument. + * @param Argument type. + * @param Result type. + * @return Result. + * @throws IgniteCheckedException If failed. + */ + public R updateDataRow(long link, PageHandler pageHnd, S arg) throws IgniteCheckedException; + /** * @param link Row link. * @throws IgniteCheckedException If failed. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageDataRow.java index 271efdff8bd94..95d8e8148af36 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageDataRow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageDataRow.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.metastorage; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.persistence.Storable; /** @@ -64,6 +65,16 @@ public int partition() { return 0; } + /** {@inheritDoc} */ + @Override public int size() throws IgniteCheckedException { + return 4 + value().length; + } + + /** {@inheritDoc} */ + @Override public int headerSize() { + return 0; + } + /** {@inheritDoc} */ @Override public void link(long link) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java index 19a145f1019ac..00db5cd791163 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageTree.java @@ -79,7 +79,7 @@ public MetastorageTree(int cacheId, } /** {@inheritDoc} */ - @Override protected MetastorageDataRow getRow(BPlusIO io, long pageAddr, int idx, + @Override public MetastorageDataRow getRow(BPlusIO io, long pageAddr, int idx, Object x) throws IgniteCheckedException { long link = ((DataLinkIO)io).getLink(pageAddr, idx); String key = ((DataLinkIO)io).getKey(pageAddr, idx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java index 8a63a00a652a1..125bc35657931 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java @@ -930,12 +930,13 @@ private long getFirstPageId(long metaId, long metaPage, int lvl) { /** * @param upper Upper bound. + * @param c Filter closure. * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. * @return Cursor. * @throws IgniteCheckedException If failed. */ - private GridCursor findLowerUnbounded(L upper, Object x) throws IgniteCheckedException { - ForwardCursor cursor = new ForwardCursor(null, upper, x); + private GridCursor findLowerUnbounded(L upper, TreeRowClosure c, Object x) throws IgniteCheckedException { + ForwardCursor cursor = new ForwardCursor(null, upper, c, x); long firstPageId; @@ -953,7 +954,7 @@ private GridCursor findLowerUnbounded(L upper, Object x) throws IgniteChecked long pageAddr = readLock(firstPageId, firstPage); // We always merge pages backwards, the first page is never removed. try { - cursor.init(pageAddr, io(pageAddr), 0); + cursor.init(pageAddr, io(pageAddr), -1); } finally { readUnlock(firstPageId, firstPage, pageAddr); @@ -981,13 +982,25 @@ private void checkDestroyed() { /** {@inheritDoc} */ public final GridCursor find(L lower, L upper, Object x) throws IgniteCheckedException { + return find(lower, upper, null, x); + } + + /** + * @param lower Lower bound inclusive or {@code null} if unbounded. + * @param upper Upper bound inclusive or {@code null} if unbounded. + * @param c Filter closure. + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + * @return Cursor. + * @throws IgniteCheckedException If failed. + */ + public final GridCursor find(L lower, L upper, TreeRowClosure c, Object x) throws IgniteCheckedException { checkDestroyed(); try { if (lower == null) - return findLowerUnbounded(upper, x); + return findLowerUnbounded(upper, c, x); - ForwardCursor cursor = new ForwardCursor(lower, upper, x); + ForwardCursor cursor = new ForwardCursor(lower, upper, c, x); cursor.find(); @@ -1004,43 +1017,159 @@ public final GridCursor find(L lower, L upper, Object x) throws IgniteChecked } } + /** + * @param lower Lower bound inclusive. + * @param upper Upper bound inclusive. + * @param c Closure applied for all found items, iteration is stopped if closure returns {@code false}. + * @throws IgniteCheckedException If failed. + */ + public void iterate(L lower, L upper, TreeRowClosure c) throws IgniteCheckedException { + checkDestroyed(); + + try { + ClosureCursor cursor = new ClosureCursor(lower, upper, c); + + cursor.iterate(); + } + catch (IgniteCheckedException e) { + throw new IgniteCheckedException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e); + } + catch (RuntimeException e) { + throw new IgniteException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e); + } + catch (AssertionError e) { + throw new AssertionError("Assertion error on bounds: [lower=" + lower + ", upper=" + upper + "]", e); + } + finally { + checkDestroyed(); + } + } + + /** + * @param lower Lower bound inclusive. + * @param upper Upper bound inclusive. + * @param c Closure applied for all found items. + * @throws IgniteCheckedException If failed. + */ + public void visit(L lower, L upper, TreeVisitorClosure c) throws IgniteCheckedException { + checkDestroyed(); + + try { + new TreeVisitor(lower, upper, c).visit(); + } + catch (IgniteCheckedException e) { + throw new IgniteCheckedException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e); + } + catch (RuntimeException e) { + throw new IgniteException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e); + } + catch (AssertionError e) { + throw new AssertionError("Assertion error on bounds: [lower=" + lower + ", upper=" + upper + "]", e); + } + finally { + checkDestroyed(); + } + } + /** {@inheritDoc} */ @Override public T findFirst() throws IgniteCheckedException { + return findFirst(null); + } + + /** + * Returns a value mapped to the lowest key, or {@code null} if tree is empty or no entry matches the passed filter. + * @param filter Filter closure. + * @return Value. + * @throws IgniteCheckedException If failed. + */ + public T findFirst(TreeRowClosure filter) throws IgniteCheckedException { checkDestroyed(); try { - long firstPageId; - - long metaPage = acquirePage(metaPageId); - try { - firstPageId = getFirstPageId(metaPageId, metaPage, 0); - } - finally { - releasePage(metaPageId, metaPage); - } + for (;;) { + long curPageId; - long page = acquirePage(firstPageId); + long metaPage = acquirePage(metaPageId); - try { - long pageAddr = readLock(firstPageId, page); + try { + curPageId = getFirstPageId(metaPageId, metaPage, 0); // Level 0 is always at the bottom. + } + finally { + releasePage(metaPageId, metaPage); + } + long curPage = acquirePage(curPageId); try { - BPlusIO io = io(pageAddr); + long curPageAddr = readLock(curPageId, curPage); - int cnt = io.getCount(pageAddr); + if (curPageAddr == 0) + continue; // The first page has gone: restart scan. - if (cnt == 0) - return null; + try { + BPlusIO io = io(curPageAddr); - return getRow(io, pageAddr, 0); + assert io.isLeaf(); + + for (;;) { + int cnt = io.getCount(curPageAddr); + + for (int i = 0; i < cnt; ++i) { + if (filter == null || filter.apply(this, io, curPageAddr, i)) + return getRow(io, curPageAddr, i); + } + + long nextPageId = io.getForward(curPageAddr); + + if (nextPageId == 0) + return null; + + long nextPage = acquirePage(nextPageId); + + try { + long nextPageAddr = readLock(nextPageId, nextPage); + + // In the current implementation the next page can't change when the current page is locked. + assert nextPageAddr != 0 : nextPageAddr; + + try { + long pa = curPageAddr; + curPageAddr = 0; // Set to zero to avoid double unlocking in finalizer. + + readUnlock(curPageId, curPage, pa); + + long p = curPage; + curPage = 0; // Set to zero to avoid double release in finalizer. + + releasePage(curPageId, p); + + curPageId = nextPageId; + curPage = nextPage; + curPageAddr = nextPageAddr; + + nextPage = 0; + nextPageAddr = 0; + } + finally { + if (nextPageAddr != 0) + readUnlock(nextPageId, nextPage, nextPageAddr); + } + } + finally { + if (nextPage != 0) + releasePage(nextPageId, nextPage); + } + } + } + finally { + if (curPageAddr != 0) + readUnlock(curPageId, curPage, curPageAddr); + } } finally { - readUnlock(firstPageId, page, pageAddr); + if (curPage != 0) + releasePage(curPageId, curPage); } } - finally { - releasePage(firstPageId, page); - } } catch (IgniteCheckedException e) { throw new IgniteCheckedException("Runtime failure on first row lookup", e); @@ -1056,13 +1185,26 @@ public final GridCursor find(L lower, L upper, Object x) throws IgniteChecked /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public T findLast() throws IgniteCheckedException { + return findLast(null); + } + + /** + * Returns a value mapped to the greatest key, or {@code null} if tree is empty or no entry matches the passed filter. + * @param c Filter closure. + * @return Value. + * @throws IgniteCheckedException If failed. + */ + public T findLast(final TreeRowClosure c) throws IgniteCheckedException { checkDestroyed(); try { - GetOne g = new GetOne(null, null, true); - doFind(g); + if (c == null) { + GetOne g = new GetOne(null, null, null, true); + doFind(g); - return (T)g.row; + return (T)g.row; + } else + return new GetLast(c).find(); } catch (IgniteCheckedException e) { throw new IgniteCheckedException("Runtime failure on last row lookup", e); @@ -1075,6 +1217,16 @@ public final GridCursor find(L lower, L upper, Object x) throws IgniteChecked } } + /** + * @param row Lookup row for exact match. + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + * @return Found result or {@code null} + * @throws IgniteCheckedException If failed. + */ + public final R findOne(L row, Object x) throws IgniteCheckedException { + return findOne(row, null, x); + } + /** * @param row Lookup row for exact match. * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. @@ -1082,11 +1234,11 @@ public final GridCursor find(L lower, L upper, Object x) throws IgniteChecked * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") - public final R findOne(L row, Object x) throws IgniteCheckedException { + public final R findOne(L row, TreeRowClosure c, Object x) throws IgniteCheckedException { checkDestroyed(); try { - GetOne g = new GetOne(row, x, false); + GetOne g = new GetOne(row, c, x, false); doFind(g); @@ -1110,7 +1262,7 @@ public final R findOne(L row, Object x) throws IgniteCheckedException { */ @SuppressWarnings("unchecked") @Override public final T findOne(L row) throws IgniteCheckedException { - return findOne(row, null); + return findOne(row, null, null); } /** @@ -2388,6 +2540,88 @@ private Result putDown(final Put p, final long pageId, final long fwdId, final i } } + + /** + * @param c Get. + * @throws IgniteCheckedException If failed. + */ + private void doVisit(TreeVisitor c) throws IgniteCheckedException { + for (;;) { // Go down with retries. + c.init(); + + switch (visitDown(c, c.rootId, 0L, c.rootLvl)) { + case RETRY: + case RETRY_ROOT: + checkInterrupted(); + + continue; + + default: + return; + } + } + } + + /** + * @param v Tree visitor. + * @param pageId Page ID. + * @param fwdId Expected forward page ID. + * @param lvl Level. + * @return Result code. + * @throws IgniteCheckedException If failed. + */ + private Result visitDown(final TreeVisitor v, final long pageId, final long fwdId, final int lvl) + throws IgniteCheckedException { + long page = acquirePage(pageId); + + try { + for (;;) { + v.checkLockRetry(); + + // Init args. + v.pageId = pageId; + v.fwdId = fwdId; + + Result res = read(pageId, page, search, v, lvl, RETRY); + + switch (res) { + case GO_DOWN: + case GO_DOWN_X: + assert v.pageId != pageId; + assert v.fwdId != fwdId || fwdId == 0; + + // Go down recursively. + res = visitDown(v, v.pageId, v.fwdId, lvl - 1); + + switch (res) { + case RETRY: + checkInterrupted(); + + continue; // The child page got split, need to reread our page. + + default: + return res; + } + + case NOT_FOUND: + assert lvl == 0 : lvl; + + return v.init(pageId, page, fwdId); + + case FOUND: + throw new IllegalStateException(); // Must never be called because we always have a shift. + + default: + return res; + } + } + } + finally { + if (v.canRelease(pageId, lvl)) + releasePage(pageId, page); + } + } + /** * @param io IO. * @param pageAddr Page address. @@ -2602,15 +2836,20 @@ private final class GetOne extends Get { /** */ Object x; + /** */ + TreeRowClosure c; + /** * @param row Row. + * @param c Closure filter. * @param x Implementation specific argument. * @param findLast Ignore row passed, find last row */ - private GetOne(L row, Object x, boolean findLast) { + private GetOne(L row, TreeRowClosure c, Object x, boolean findLast) { super(row, findLast); this.x = x; + this.c = c; } /** {@inheritDoc} */ @@ -2620,7 +2859,7 @@ private GetOne(L row, Object x, boolean findLast) { if (lvl != 0 && !canGetRowFromInner) return false; - row = getRow(io, pageAddr, idx, x); + row = c == null || c.apply(BPlusTree.this, io,pageAddr, idx) ? getRow(io, pageAddr, idx, x) : null; return true; } @@ -2631,14 +2870,14 @@ private GetOne(L row, Object x, boolean findLast) { */ private final class GetCursor extends Get { /** */ - ForwardCursor cursor; + AbstractForwardCursor cursor; /** * @param lower Lower bound. * @param shift Shift. * @param cursor Cursor. */ - GetCursor(L lower, int shift, ForwardCursor cursor) { + GetCursor(L lower, int shift, AbstractForwardCursor cursor) { super(lower, false); assert shift != 0; // Either handle range of equal rows or find a greater row after concurrent merge. @@ -2664,97 +2903,452 @@ private final class GetCursor extends Get { } /** - * Put operation. + * Get a cursor for range. */ - public final class Put extends Get { - /** Mark of NULL value of page id. It means valid value can't be equal this value. */ - private static final long NULL_PAGE_ID = 0L; - - /** Mark of NULL value of page. */ - private static final long NULL_PAGE = 0L; - - /** Mark of NULL value of page address. */ - private static final long NULL_PAGE_ADDRESS = 0L; - - /** Right child page ID for split row. */ - long rightId; - - /** Replaced row if any. */ - T oldRow; - - /** - * This page is kept locked after split until insert to the upper level will not be finished. It is needed - * because split row will be "in flight" and if we'll release tail, remove on split row may fail. - */ - long tailId; - + private final class TreeVisitor extends Get { /** */ - long tailPage; + long nextPageId; /** */ - long tailAddr; + L upper; - /** - * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level. - */ - short btmLvl; + /** */ + TreeVisitorClosure p; /** */ - Bool needReplaceInner = FALSE; + private boolean dirty; /** */ - final boolean needOld; + private boolean writing; /** - * @param row Row. - * @param needOld {@code True} If need return old value. + * @param lower Lower bound. */ - private Put(T row, boolean needOld) { - super(row, false); + TreeVisitor(L lower, L upper, TreeVisitorClosure p) { + super(lower, false); - this.needOld = needOld; + this.shift = -1; + this.upper = upper; + this.p = p; } /** {@inheritDoc} */ - @Override boolean found(BPlusIO io, long pageAddr, int idx, int lvl) { - if (lvl == 0) // Leaf: need to stop. - return true; + @Override boolean found(BPlusIO io, long pageAddr, int idx, int lvl) throws IgniteCheckedException { + throw new IllegalStateException(); // Must never be called because we always have a shift. + } - assert btmLvl == 0; // It can not be insert. + /** {@inheritDoc} */ + @Override boolean notFound(BPlusIO io, long pageAddr, int idx, int lvl) throws IgniteCheckedException { + if (lvl != 0) + return false; - // If we can get full row from the inner page, we have to replace it with the new one. On the way down - // we can not miss inner key even in presence of concurrent operations because of `triangle` invariant + - // concurrent inner replace handling by retrying from root. - if (canGetRowFromInner && needReplaceInner == FALSE) - needReplaceInner = TRUE; + if (!(writing = (p.state() & TreeVisitorClosure.CAN_WRITE) != 0)) + init(pageAddr, io, idx); - return false; + return true; } - /** {@inheritDoc} */ - @Override boolean notFound(BPlusIO io, long pageAddr, int idx, int lvl) { - assert btmLvl >= 0 : btmLvl; - assert lvl >= btmLvl : lvl; + Result init(long pageId, long page, long fwdId) throws IgniteCheckedException { + // Init args. + this.pageId = pageId; + this.fwdId = fwdId; - return lvl == btmLvl; - } + if (writing) { + long pageAddr = writeLock(pageId, page); - /** - * @param tailId Tail page ID. - * @param tailPage Tail page pointer. - * @param tailPageAddr Tail page address - */ - private void tail(long tailId, long tailPage, long tailPageAddr) { - assert (tailId == NULL_PAGE_ID) == (tailPage == NULL_PAGE); - assert (tailPage == NULL_PAGE) == (tailPageAddr == NULL_PAGE_ADDRESS); + if (pageAddr == 0) + return RETRY; - if (this.tailPage != NULL_PAGE) - writeUnlockAndClose(this.tailId, this.tailPage, this.tailAddr, null); + try { + BPlusIO io = io(pageAddr); - this.tailId = tailId; - this.tailPage = tailPage; - this.tailAddr = tailPageAddr; - } + // Check triangle invariant. + if (io.getForward(pageAddr) != fwdId) + return RETRY; + + init(pageAddr, io, -1); + } finally { + unlock(pageId, page, pageAddr); + } + } + + return NOT_FOUND; + } + + /** + * @param pageAddr Page address. + * @param io IO. + * @param startIdx Start index. + * @throws IgniteCheckedException If failed. + */ + private void init(long pageAddr, BPlusIO io, int startIdx) throws IgniteCheckedException { + nextPageId = 0; + + int cnt = io.getCount(pageAddr); + + if (cnt != 0) + visit(pageAddr, io, startIdx, cnt); + } + + /** + * @param pageAddr Page address. + * @param io IO. + * @param startIdx Start index. + * @param cnt Number of rows in the buffer. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("unchecked") + private void visit(long pageAddr, BPlusIO io, int startIdx, int cnt) + throws IgniteCheckedException { + assert io.isLeaf() : io; + assert cnt != 0 : cnt; // We can not see empty pages (empty tree handled in init). + assert startIdx >= -1 : startIdx; + assert cnt >= startIdx; + + checkDestroyed(); + + nextPageId = io.getForward(pageAddr); + + if (startIdx == -1) + startIdx = findLowerBound(pageAddr, io, cnt); + + if (cnt == startIdx) + return; // Go to the next page; + + cnt = findUpperBound(pageAddr, io, startIdx, cnt); + + for (int i = startIdx; i < cnt; i++) { + int state = p.visit(BPlusTree.this, io, pageAddr, i, wal); + + boolean stop = (state & TreeVisitorClosure.STOP) != 0; + + if (writing) + dirty = dirty || (state & TreeVisitorClosure.DIRTY) != 0; + + if (stop) { + nextPageId = 0; // The End. + + return; + } + } + + if (nextPageId != 0) { + row = io.getLookupRow(BPlusTree.this, pageAddr, cnt - 1); // Need save last row. + + shift = 1; + } + } + + /** + * @param pageAddr Page address. + * @param io IO. + * @param cnt Count. + * @return Adjusted to lower bound start index. + * @throws IgniteCheckedException If failed. + */ + private int findLowerBound(long pageAddr, BPlusIO io, int cnt) throws IgniteCheckedException { + assert io.isLeaf(); + + // Compare with the first row on the page. + int cmp = compare(0, io, pageAddr, 0, row); + + if (cmp < 0 || (cmp == 0 && shift == 1)) { + int idx = findInsertionPoint(0, io, pageAddr, 0, cnt, row, shift); + + assert idx < 0; + + return fix(idx); + } + + return 0; + } + + /** + * @param pageAddr Page address. + * @param io IO. + * @param low Start index. + * @param cnt Number of rows in the buffer. + * @return Corrected number of rows with respect to upper bound. + * @throws IgniteCheckedException If failed. + */ + private int findUpperBound(long pageAddr, BPlusIO io, int low, int cnt) throws IgniteCheckedException { + assert io.isLeaf(); + + // Compare with the last row on the page. + int cmp = compare(0, io, pageAddr, cnt - 1, upper); + + if (cmp > 0) { + int idx = findInsertionPoint(0, io, pageAddr, low, cnt, upper, 1); + + assert idx < 0; + + cnt = fix(idx); + + nextPageId = 0; // The End. + } + + return cnt; + } + + /** + * @throws IgniteCheckedException If failed. + */ + private void nextPage() throws IgniteCheckedException { + for (;;) { + if (nextPageId == 0) + return; + + long pageId = nextPageId; + long page = acquirePage(pageId); + try { + long pageAddr = lock(pageId, page); // Doing explicit null check. + + // If concurrent merge occurred we have to reinitialize cursor from the last returned row. + if (pageAddr == 0L) + break; + + try { + BPlusIO io = io(pageAddr); + + visit(pageAddr, io, -1, io.getCount(pageAddr)); + } + finally { + unlock(pageId, page, pageAddr); + } + } + finally { + releasePage(pageId, page); + } + } + + doVisit(this); // restart from last read row + } + + private void unlock(long pageId, long page, long pageAddr) { + if (writing) { + writeUnlock(pageId, page, pageAddr, dirty); + + dirty = false; // reset dirty flag + } + else + readUnlock(pageId, page, pageAddr); + } + + private long lock(long pageId, long page) { + if (writing = ((p.state() & TreeVisitorClosure.CAN_WRITE) != 0)) + return writeLock(pageId, page); + else + return readLock(pageId, page); + } + + /** + * @throws IgniteCheckedException If failed. + */ + private void visit() throws IgniteCheckedException { + doVisit(this); + + while (nextPageId != 0) + nextPage(); + } + } + + /** + * Get the last item in the tree which matches the passed filter. + */ + private final class GetLast extends Get { + private final TreeRowClosure c; + private boolean retry = true; + private long lastPageId; + private T row0; + + /** + * @param c Filter closure. + */ + public GetLast(TreeRowClosure c) { + super(null, true); + + assert c != null; + + this.c = c; + } + + /** {@inheritDoc} */ + @Override boolean found(BPlusIO io, long pageAddr, int idx, int lvl) throws IgniteCheckedException { + if (lvl != 0) + return false; + + for (int i = idx; i >= 0; i--) { + if (c.apply(BPlusTree.this, io, pageAddr, i)) { + retry = false; + row0 = getRow(io, pageAddr, i); + + return true; + } + } + + if(pageId == rootId) + retry = false; // We are at the root page, there are no other leafs. + + if (retry) { + findLast = false; + + // Restart from an item before the first item in the leaf (last item on the previous leaf). + row0 = getRow(io, pageAddr, 0); + shift = -1; + + lastPageId = pageId; // Track leafs to detect a loop over the first leaf in the tree. + } + + return true; + } + + /** {@inheritDoc} */ + @Override boolean notFound(BPlusIO io, long pageAddr, int idx, int lvl) throws IgniteCheckedException { + if (lvl != 0) + return false; + + if(io.getCount(pageAddr) == 0) { + // it's an empty tree + retry = false; + + return true; + } + + if (idx == 0 && lastPageId == pageId) { + // not found + retry = false; + row0 = null; + + return true; + } + else { + for (int i = idx; i >= 0; i--) { + if (c.apply(BPlusTree.this, io, pageAddr, i)) { + retry = false; + row0 = getRow(io, pageAddr, i); + + break; + } + } + } + + if (retry) { + // Restart from an item before the first item in the leaf (last item on the previous leaf). + row0 = getRow(io, pageAddr, 0); + + lastPageId = pageId; // Track leafs to detect a loop over the first leaf in the tree. + } + + return true; + } + + /** + * @return Last item in the tree. + * @throws IgniteCheckedException If failure. + */ + public T find() throws IgniteCheckedException { + while (retry) { + row = row0; + + doFind(this); + } + + return row0; + } + } + + /** + * Put operation. + */ + public final class Put extends Get { + /** Mark of NULL value of page id. It means valid value can't be equal this value. */ + private static final long NULL_PAGE_ID = 0L; + + /** Mark of NULL value of page. */ + private static final long NULL_PAGE = 0L; + + /** Mark of NULL value of page address. */ + private static final long NULL_PAGE_ADDRESS = 0L; + + /** Right child page ID for split row. */ + long rightId; + + /** Replaced row if any. */ + T oldRow; + + /** + * This page is kept locked after split until insert to the upper level will not be finished. It is needed + * because split row will be "in flight" and if we'll release tail, remove on split row may fail. + */ + long tailId; + + /** */ + long tailPage; + + /** */ + long tailAddr; + + /** + * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level. + */ + short btmLvl; + + /** */ + Bool needReplaceInner = FALSE; + + /** */ + final boolean needOld; + + /** + * @param row Row. + * @param needOld {@code True} If need return old value. + */ + private Put(T row, boolean needOld) { + super(row, false); + + this.needOld = needOld; + } + + /** {@inheritDoc} */ + @Override boolean found(BPlusIO io, long pageAddr, int idx, int lvl) { + if (lvl == 0) // Leaf: need to stop. + return true; + + assert btmLvl == 0; // It can not be insert. + + // If we can get full row from the inner page, we have to replace it with the new one. On the way down + // we can not miss inner key even in presence of concurrent operations because of `triangle` invariant + + // concurrent inner replace handling by retrying from root. + if (canGetRowFromInner && needReplaceInner == FALSE) + needReplaceInner = TRUE; + + return false; + } + + /** {@inheritDoc} */ + @Override boolean notFound(BPlusIO io, long pageAddr, int idx, int lvl) { + assert btmLvl >= 0 : btmLvl; + assert lvl >= btmLvl : lvl; + + return lvl == btmLvl; + } + + /** + * @param tailId Tail page ID. + * @param tailPage Tail page pointer. + * @param tailPageAddr Tail page address + */ + private void tail(long tailId, long tailPage, long tailPageAddr) { + assert (tailId == NULL_PAGE_ID) == (tailPage == NULL_PAGE); + assert (tailPage == NULL_PAGE) == (tailPageAddr == NULL_PAGE_ADDRESS); + + if (this.tailPage != NULL_PAGE) + writeUnlockAndClose(this.tailId, this.tailPage, this.tailAddr, null); + + this.tailId = tailId; + this.tailPage = tailPage; + this.tailAddr = tailPageAddr; + } /** {@inheritDoc} */ @Override public boolean canRelease(long pageId, int lvl) { @@ -4520,7 +5114,7 @@ protected int compare(int lvl, BPlusIO io, long pageAddr, int idx, L row) thr * @return Full detached data row. * @throws IgniteCheckedException If failed. */ - protected final T getRow(BPlusIO io, long pageAddr, int idx) throws IgniteCheckedException { + public final T getRow(BPlusIO io, long pageAddr, int idx) throws IgniteCheckedException { return getRow(io, pageAddr, idx, null); } @@ -4534,54 +5128,60 @@ protected final T getRow(BPlusIO io, long pageAddr, int idx) throws IgniteChe * @return Data row. * @throws IgniteCheckedException If failed. */ - protected abstract T getRow(BPlusIO io, long pageAddr, int idx, Object x) throws IgniteCheckedException; + public abstract T getRow(BPlusIO io, long pageAddr, int idx, Object x) throws IgniteCheckedException; /** - * Forward cursor. + * */ @SuppressWarnings("unchecked") - private final class ForwardCursor implements GridCursor { + private abstract class AbstractForwardCursor { /** */ - private T[] rows = (T[])EMPTY; - - /** */ - private int row = -1; + long nextPageId; /** */ - private long nextPageId; - - /** */ - private L lowerBound; + L lowerBound; /** */ private int lowerShift = -1; // Initially it is -1 to handle multiple equal rows. /** */ - private final L upperBound; - - /** */ - private final Object x; + final L upperBound; /** * @param lowerBound Lower bound. * @param upperBound Upper bound. */ - ForwardCursor(L lowerBound, L upperBound) { + AbstractForwardCursor(L lowerBound, L upperBound) { this.lowerBound = lowerBound; this.upperBound = upperBound; - this.x = null; } /** - * @param lowerBound Lower bound. - * @param upperBound Upper bound. - * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + * */ - ForwardCursor(L lowerBound, L upperBound, Object x) { - this.lowerBound = lowerBound; - this.upperBound = upperBound; - this.x = x; - } + abstract void init0(); + + /** + * @param pageAddr Page address. + * @param io IO. + * @param startIdx Start index. + * @param cnt Number of rows in the buffer. + * @return {@code true} If we were able to fetch rows from this page. + * @throws IgniteCheckedException If failed. + */ + abstract boolean fillFromBuffer0(long pageAddr, BPlusIO io, int startIdx, int cnt) + throws IgniteCheckedException; + + /** + * @return {@code True} If we have rows to return after reading the next page. + * @throws IgniteCheckedException If failed. + */ + abstract boolean reinitialize0() throws IgniteCheckedException; + + /** + * @param readDone {@code True} if traversed all rows. + */ + abstract void onNotFound(boolean readDone); /** * @param pageAddr Page address. @@ -4589,9 +5189,10 @@ private final class ForwardCursor implements GridCursor { * @param startIdx Start index. * @throws IgniteCheckedException If failed. */ - private void init(long pageAddr, BPlusIO io, int startIdx) throws IgniteCheckedException { + final void init(long pageAddr, BPlusIO io, int startIdx) throws IgniteCheckedException { nextPageId = 0; - row = -1; + + init0(); int cnt = io.getCount(pageAddr); @@ -4599,16 +5200,10 @@ private void init(long pageAddr, BPlusIO io, int startIdx) throws IgniteCheck if (cnt == 0) { assert io.getForward(pageAddr) == 0L; - rows = null; - } - else if (!fillFromBuffer(pageAddr, io, startIdx, cnt)) { - if (rows != EMPTY) { - assert rows.length > 0; // Otherwise it makes no sense to create an array. - - // Fake clear. - rows[0] = null; - } + onNotFound(true); } + else if (!fillFromBuffer(pageAddr, io, startIdx, cnt)) + onNotFound(false); } /** @@ -4618,7 +5213,7 @@ else if (!fillFromBuffer(pageAddr, io, startIdx, cnt)) { * @return Adjusted to lower bound start index. * @throws IgniteCheckedException If failed. */ - private int findLowerBound(long pageAddr, BPlusIO io, int cnt) throws IgniteCheckedException { + final int findLowerBound(long pageAddr, BPlusIO io, int cnt) throws IgniteCheckedException { assert io.isLeaf(); // Compare with the first row on the page. @@ -4643,7 +5238,7 @@ private int findLowerBound(long pageAddr, BPlusIO io, int cnt) throws IgniteC * @return Corrected number of rows with respect to upper bound. * @throws IgniteCheckedException If failed. */ - private int findUpperBound(long pageAddr, BPlusIO io, int low, int cnt) throws IgniteCheckedException { + final int findUpperBound(long pageAddr, BPlusIO io, int low, int cnt) throws IgniteCheckedException { assert io.isLeaf(); // Compare with the last row on the page. @@ -4675,84 +5270,20 @@ private boolean fillFromBuffer(long pageAddr, BPlusIO io, int startIdx, int c throws IgniteCheckedException { assert io.isLeaf() : io; assert cnt != 0 : cnt; // We can not see empty pages (empty tree handled in init). - assert startIdx >= 0 : startIdx; + assert startIdx >= 0 || startIdx == -1: startIdx; assert cnt >= startIdx; checkDestroyed(); nextPageId = io.getForward(pageAddr); - if (lowerBound != null && startIdx == 0) - startIdx = findLowerBound(pageAddr, io, cnt); - - if (upperBound != null && cnt != startIdx) - cnt = findUpperBound(pageAddr, io, startIdx, cnt); - - cnt -= startIdx; - - if (cnt == 0) - return false; - - if (rows == EMPTY) - rows = (T[])new Object[cnt]; - - int foundCnt = 0; - - for (int i = 0; i < cnt; i++) { - T r = getRow(io, pageAddr, startIdx + i, x); - - if (r != null) - rows = GridArrays.set(rows, foundCnt++, r); - } - - if (foundCnt == 0) { - rows = (T[])EMPTY; - - return false; - } - - GridArrays.clearTail(rows, foundCnt); - - return true; - } - - /** {@inheritDoc} */ - @SuppressWarnings("SimplifiableIfStatement") - @Override public boolean next() throws IgniteCheckedException { - if (rows == null) - return false; - - if (++row < rows.length && rows[row] != null) { - clearLastRow(); // Allow to GC the last returned row. - - return true; - } - - return nextPage(); - } - - /** - * @return Cleared last row. - */ - private T clearLastRow() { - if (row == 0) - return null; - - int last = row - 1; - - T r = rows[last]; - - assert r != null; - - rows[last] = null; - - return r; + return fillFromBuffer0(pageAddr, io, startIdx, cnt); } /** * @throws IgniteCheckedException If failed. */ - private void find() throws IgniteCheckedException { + final void find() throws IgniteCheckedException { assert lowerBound != null; doFind(new GetCursor(lowerBound, lowerShift, this)); @@ -4768,21 +5299,20 @@ private boolean reinitialize() throws IgniteCheckedException { // to the previous lower bound. find(); - return next(); + return reinitialize0(); } /** + * @param lastRow Last read row (to be used as new lower bound). * @return {@code true} If we have rows to return after reading the next page. * @throws IgniteCheckedException If failed. */ - private boolean nextPage() throws IgniteCheckedException { - updateLowerBound(clearLastRow()); - - row = 0; + final boolean nextPage(L lastRow) throws IgniteCheckedException { + updateLowerBound(lastRow); for (;;) { if (nextPageId == 0) { - rows = null; + onNotFound(true); return false; // Done. } @@ -4799,7 +5329,7 @@ private boolean nextPage() throws IgniteCheckedException { try { BPlusIO io = io(pageAddr); - if (fillFromBuffer(pageAddr, io, 0, io.getCount(pageAddr))) + if (fillFromBuffer(pageAddr, io, -1, io.getCount(pageAddr))) return true; // Continue fetching forward. @@ -4820,12 +5350,239 @@ private boolean nextPage() throws IgniteCheckedException { /** * @param lower New exact lower bound. */ - private void updateLowerBound(T lower) { + private void updateLowerBound(L lower) { if (lower != null) { lowerShift = 1; // Now we have the full row an need to avoid duplicates. lowerBound = lower; // Move the lower bound forward for further concurrent merge retries. } } + } + + /** + * Closure cursor. + */ + @SuppressWarnings("unchecked") + private final class ClosureCursor extends AbstractForwardCursor { + /** */ + private final TreeRowClosure p; + + /** */ + private L lastRow; + + /** + * @param lowerBound Lower bound. + * @param upperBound Upper bound. + * @param p Row predicate. + */ + ClosureCursor(L lowerBound, L upperBound, TreeRowClosure p) { + super(lowerBound, upperBound); + + assert lowerBound != null; + assert upperBound != null; + assert p != null; + + this.p = p; + } + + /** {@inheritDoc} */ + @Override void init0() { + // No-op. + } + + /** {@inheritDoc} */ + @Override boolean fillFromBuffer0(long pageAddr, BPlusIO io, int startIdx, int cnt) + throws IgniteCheckedException { + if (startIdx == -1) + startIdx = findLowerBound(pageAddr, io, cnt); + + if (cnt == startIdx) + return false; + + for (int i = startIdx; i < cnt; i++) { + int cmp = compare(0, io, pageAddr, i, upperBound); + + if (cmp > 0) { + nextPageId = 0; // The End. + + return false; + } + + boolean stop = !p.apply(BPlusTree.this, io, pageAddr, i); + + if (stop) { + nextPageId = 0; // The End. + + return true; + } + } + + if (nextPageId != 0) + lastRow = io.getLookupRow(BPlusTree.this, pageAddr, cnt - 1); // Need save last row. + + return true; + } + + /** {@inheritDoc} */ + @Override boolean reinitialize0() throws IgniteCheckedException { + return true; + } + + /** {@inheritDoc} */ + @Override void onNotFound(boolean readDone) { + nextPageId = 0; + } + + /** + * @throws IgniteCheckedException If failed. + */ + private void iterate() throws IgniteCheckedException { + find(); + + if (nextPageId == 0) { + return; + } + + for (;;) { + L lastRow0 = lastRow; + + lastRow = null; + + nextPage(lastRow0); + + if (nextPageId == 0) + return; + } + } + } + + /** + * Forward cursor. + */ + @SuppressWarnings("unchecked") + private final class ForwardCursor extends AbstractForwardCursor implements GridCursor { + /** */ + final Object x; + + /** */ + private T[] rows = (T[])EMPTY; + + /** */ + private int row = -1; + + /** */ + private final TreeRowClosure c; + + /** + * @param lowerBound Lower bound. + * @param upperBound Upper bound. + * @param c Filter closure. + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + */ + ForwardCursor(L lowerBound, L upperBound, TreeRowClosure c, Object x) { + super(lowerBound, upperBound); + + this.c = c; + this.x = x; + } + + /** {@inheritDoc} */ + @Override boolean fillFromBuffer0(long pageAddr, BPlusIO io, int startIdx, int cnt) throws IgniteCheckedException { + if (startIdx == -1) { + if (lowerBound != null) + startIdx = findLowerBound(pageAddr, io, cnt); + else + startIdx = 0; + } + + if (upperBound != null && cnt != startIdx) + cnt = findUpperBound(pageAddr, io, startIdx, cnt); + + int cnt0 = cnt - startIdx; + + if (cnt0 == 0) + return false; + + if (rows == EMPTY) + rows = (T[])new Object[cnt0]; + + int resCnt = 0; + + for (int idx = startIdx; idx < cnt; idx++) { + if (c == null || c.apply(BPlusTree.this, io, pageAddr, idx)) + rows = GridArrays.set(rows, resCnt++, getRow(io, pageAddr, idx, x)); + } + + if (resCnt == 0) { + rows = (T[])EMPTY; + + return false; + } + + GridArrays.clearTail(rows, resCnt); + + return true; + } + + /** {@inheritDoc} */ + @Override boolean reinitialize0() throws IgniteCheckedException { + return next(); + } + + /** {@inheritDoc} */ + @Override void onNotFound(boolean readDone) { + if (readDone) + rows = null; + else { + if (rows != EMPTY) { + assert rows.length > 0; // Otherwise it makes no sense to create an array. + + // Fake clear. + rows[0] = null; + } + } + } + + /** {@inheritDoc} */ + @Override void init0() { + row = -1; + } + + /** {@inheritDoc} */ + @SuppressWarnings("SimplifiableIfStatement") + @Override public boolean next() throws IgniteCheckedException { + if (rows == null) + return false; + + if (++row < rows.length && rows[row] != null) { + clearLastRow(); // Allow to GC the last returned row. + + return true; + } + + T lastRow = clearLastRow(); + + row = 0; + + return nextPage(lastRow); + } + + /** + * @return Cleared last row. + */ + private T clearLastRow() { + if (row == 0) + return null; + + int last = row - 1; + + T r = rows[last]; + + assert r != null; + + rows[last] = null; + + return r; + } /** {@inheritDoc} */ @Override public T get() { @@ -4961,6 +5718,36 @@ public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, int idx throws IgniteCheckedException; } + /** + * A generic visitor-style interface for performing inspection/modification operations on the tree. + */ + public interface TreeVisitorClosure { + /** */ + int STOP = 0x01; + /** */ + int CAN_WRITE = STOP << 1; + /** */ + int DIRTY = CAN_WRITE << 1; + + /** + * Performs inspection or operation on a specified row. + * + * @param tree The tree. + * @param io Th tree IO object. + * @param pageAddr The page address. + * @param idx The item index. + * @return state bitset. + * @throws IgniteCheckedException If failed. + */ + public int visit(BPlusTree tree, BPlusIO io, long pageAddr, int idx, IgniteWriteAheadLogManager wal) + throws IgniteCheckedException; + + /** + * @return state bitset. + */ + public int state(); + } + /** * @return Return number of retries. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java index 50b577978e4ef..4a12045d26542 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java @@ -81,6 +81,7 @@ public abstract class AbstractDataPageIO extends PageIO { public static final int MIN_DATA_PAGE_OVERHEAD = ITEMS_OFF + ITEM_SIZE + PAYLOAD_LEN_SIZE + LINK_SIZE; /** + * @param type Page type. * @param ver Page format version. */ protected AbstractDataPageIO(int type, int ver) { @@ -500,6 +501,23 @@ public DataPagePayload readPayload(final long pageAddr, final int itemId, final nextLink); } + /** + * @param pageAddr Page address. + * @param itemId Item to position on. + * @param pageSize Page size. + * @param reqLen Required payload length. + * @return Offset to start of actual fragment data. + */ + public int getPayloadOffset(final long pageAddr, final int itemId, final int pageSize, int reqLen) { + int dataOff = getDataOffset(pageAddr, itemId, pageSize); + + int payloadSize = getPageEntrySize(pageAddr, dataOff, 0); + + assert payloadSize >= reqLen : payloadSize; + + return dataOff + PAYLOAD_LEN_SIZE + (isFragmented(pageAddr, dataOff) ? LINK_SIZE : 0); + } + /** * @param pageAddr Page address. * @param idx Item index. @@ -982,6 +1000,16 @@ private int addRowFragment( int payloadSize = payload != null ? payload.length : Math.min(rowSize - written, getFreeSpace(pageAddr)); + if (row != null) { + int remain = rowSize - written - payloadSize; + int hdrSize = row.headerSize(); + + // We need page header (i.e. MVCC info) is located entirely on the very first page in chain. + // So we force moving it to the next page if it could not fit entirely on this page. + if (remain > 0 && remain < hdrSize) + payloadSize -= hdrSize - remain; + } + int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM); int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize); @@ -1226,13 +1254,6 @@ protected void writeRowData( PageUtils.putBytes(pageAddr, dataOff, payload); } - /** - * @param row Row. - * @return Row size in page. - * @throws IgniteCheckedException if failed. - */ - public abstract int getRowSize(T row) throws IgniteCheckedException; - /** * Defines closure interface for applying computations to data page items. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java index 8a04749a32bb3..87e37f6c224a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java @@ -21,15 +21,26 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.processors.cache.CacheObject; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridStringBuilder; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.EntryPart.CACHE_ID; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.EntryPart.EXPIRE_TIME; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.EntryPart.KEY; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.EntryPart.MVCC_INFO; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.EntryPart.VALUE; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.EntryPart.VERSION; + /** * Data pages IO. */ public class DataPageIO extends AbstractDataPageIO { + /** */ + public static final int MVCC_INFO_SIZE = 40; + /** */ public static final IOVersions VERSIONS = new IOVersions<>( new DataPageIO(1) @@ -43,21 +54,74 @@ protected DataPageIO(int ver) { } /** {@inheritDoc} */ - @Override - protected void writeFragmentData( - final CacheDataRow row, - final ByteBuffer buf, - final int rowOff, - final int payloadSize - ) throws IgniteCheckedException { + @Override protected void writeRowData(long pageAddr, int dataOff, int payloadSize, CacheDataRow row, + boolean newRow) throws IgniteCheckedException { + long addr = pageAddr + dataOff; + + int cacheIdSize = row.cacheId() != 0 ? 4 : 0; + int mvccInfoSize = row.mvccCoordinatorVersion() > 0 ? MVCC_INFO_SIZE : 0; + + if (newRow) { + PageUtils.putShort(addr, 0, (short)payloadSize); + addr += 2; + + if (mvccInfoSize > 0) { + assert MvccUtils.mvccVersionIsValid(row.mvccCoordinatorVersion(), row.mvccCounter(), row.mvccOperationCounter()); + + // xid_min. + PageUtils.putLong(addr, 0, row.mvccCoordinatorVersion()); + PageUtils.putLong(addr, 8, row.mvccCounter()); + PageUtils.putInt(addr, 16, row.mvccOperationCounter() | (row.mvccTxState() << MVCC_HINTS_BIT_OFF)); + + assert row.newMvccCoordinatorVersion() == 0 + || MvccUtils.mvccVersionIsValid(row.newMvccCoordinatorVersion(), row.newMvccCounter(), row.newMvccOperationCounter()); + + // xid_max. + PageUtils.putLong(addr, 20, row.newMvccCoordinatorVersion()); + PageUtils.putLong(addr, 28, row.newMvccCounter()); + PageUtils.putInt(addr, 36, row.newMvccOperationCounter() | (row.newMvccTxState() << MVCC_HINTS_BIT_OFF)); + + addr += mvccInfoSize; + } + + if (cacheIdSize != 0) { + PageUtils.putInt(addr, 0, row.cacheId()); + + addr += cacheIdSize; + } + + addr += row.key().putValue(addr); + } + else + addr += (2 + mvccInfoSize + cacheIdSize + row.key().valueBytesLength(null)); + + addr += row.value().putValue(addr); + + CacheVersionIO.write(addr, row.version(), false); + addr += CacheVersionIO.size(row.version(), false); + + PageUtils.putLong(addr, 0, row.expireTime()); + } + + /** {@inheritDoc} */ + @Override protected void writeFragmentData(CacheDataRow row, ByteBuffer buf, int rowOff, + int payloadSize) throws IgniteCheckedException { final int keySize = row.key().valueBytesLength(null); + final int valSize = row.value().valueBytesLength(null); - int written = writeFragment(row, buf, rowOff, payloadSize, EntryPart.CACHE_ID, keySize, valSize); - written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.KEY, keySize, valSize); - written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.EXPIRE_TIME, keySize, valSize); - written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.VALUE, keySize, valSize); - written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.VERSION, keySize, valSize); + int written = writeFragment(row, buf, rowOff, payloadSize, + MVCC_INFO, keySize, valSize); + written += writeFragment(row, buf, rowOff + written, payloadSize - written, + CACHE_ID, keySize, valSize); + written += writeFragment(row, buf, rowOff + written, payloadSize - written, + KEY, keySize, valSize); + written += writeFragment(row, buf, rowOff + written, payloadSize - written, + EXPIRE_TIME, keySize, valSize); + written += writeFragment(row, buf, rowOff + written, payloadSize - written, + VALUE, keySize, valSize); + written += writeFragment(row, buf, rowOff + written, payloadSize - written, + VERSION, keySize, valSize); assert written == payloadSize; } @@ -65,9 +129,13 @@ protected void writeFragmentData( /** * Try to write fragment data. * + * @param row Row. + * @param buf Byte buffer. * @param rowOff Offset in row data bytes. * @param payloadSize Data length that should be written in this fragment. * @param type Type of the part of entry. + * @param keySize Key size. + * @param valSize Value size. * @return Actually written data. * @throws IgniteCheckedException If fail. */ @@ -87,35 +155,42 @@ private int writeFragment( final int curLen; int cacheIdSize = row.cacheId() == 0 ? 0 : 4; + int mvccInfoSize = row.mvccCoordinatorVersion() > 0 ? MVCC_INFO_SIZE : 0; switch (type) { - case CACHE_ID: + case MVCC_INFO: prevLen = 0; - curLen = cacheIdSize; + curLen = mvccInfoSize; + + break; + + case CACHE_ID: + prevLen = mvccInfoSize; + curLen = mvccInfoSize + cacheIdSize; break; case KEY: - prevLen = cacheIdSize; - curLen = cacheIdSize + keySize; + prevLen = mvccInfoSize + cacheIdSize; + curLen = mvccInfoSize + cacheIdSize + keySize; break; case EXPIRE_TIME: - prevLen = cacheIdSize + keySize; - curLen = cacheIdSize + keySize + 8; + prevLen = mvccInfoSize + cacheIdSize + keySize; + curLen = mvccInfoSize + cacheIdSize + keySize + 8; break; case VALUE: - prevLen = cacheIdSize + keySize + 8; - curLen = cacheIdSize + keySize + valSize + 8; + prevLen = mvccInfoSize + cacheIdSize + keySize + 8; + curLen = mvccInfoSize + cacheIdSize + keySize + valSize + 8; break; case VERSION: - prevLen = cacheIdSize + keySize + valSize + 8; - curLen = cacheIdSize + keySize + valSize + CacheVersionIO.size(row.version(), false) + 8; + prevLen = mvccInfoSize + cacheIdSize + keySize + valSize + 8; + curLen = mvccInfoSize + cacheIdSize + keySize + valSize + CacheVersionIO.size(row.version(), false) + 8; break; @@ -128,13 +203,22 @@ private int writeFragment( final int len = Math.min(curLen - rowOff, payloadSize); - if (type == EntryPart.EXPIRE_TIME) + if (type == EXPIRE_TIME) writeExpireTimeFragment(buf, row.expireTime(), rowOff, len, prevLen); - else if (type == EntryPart.CACHE_ID) + else if (type == CACHE_ID) writeCacheIdFragment(buf, row.cacheId(), rowOff, len, prevLen); - else if (type != EntryPart.VERSION) { + else if (type == MVCC_INFO) + writeMvccInfoFragment(buf, + row.mvccCoordinatorVersion(), + row.mvccCounter(), + row.mvccOperationCounter() | (row.mvccTxState() << MVCC_HINTS_BIT_OFF), + row.newMvccCoordinatorVersion(), + row.newMvccCounter(), + row.newMvccOperationCounter() | (row.newMvccTxState() << MVCC_HINTS_BIT_OFF), + len); + else if (type != VERSION) { // Write key or value. - final CacheObject co = type == EntryPart.KEY ? row.key() : row.value(); + final CacheObject co = type == KEY ? row.key() : row.value(); co.putValue(buf, rowOff - prevLen, len); } @@ -144,6 +228,236 @@ else if (type != EntryPart.VERSION) { return len; } + /** + * @param pageAddr Page address. + * @param dataOff Data offset. + * @param ver New version. + */ + public void updateVersion(long pageAddr, int dataOff, MvccVersion ver) { + long addr = pageAddr + dataOff; + + updateVersion(addr, ver.coordinatorVersion(), ver.counter(), ver.operationCounter()); + } + + /** + * @param pageAddr Page address. + * @param itemId Item ID. + * @param pageSize Page size. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + * @param mvccOpCntr Operation counter. + */ + public void updateVersion(long pageAddr, int itemId, int pageSize, long mvccCrd, long mvccCntr, int mvccOpCntr) { + int dataOff = getDataOffset(pageAddr, itemId, pageSize); + + long addr = pageAddr + dataOff + (isFragmented(pageAddr, dataOff) ? 10 : 2); + + updateVersion(addr, mvccCrd, mvccCntr, mvccOpCntr); + } + + /** + * @param addr Address. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + */ + private void updateVersion(long addr, long mvccCrd, long mvccCntr, int mvccOpCntr) { + PageUtils.putLong(addr, 0, mvccCrd); + PageUtils.putLong(addr, 8, mvccCntr); + PageUtils.putInt(addr, 16, mvccOpCntr); + } + + /** + * @param pageAddr Page address. + * @param dataOff Data offset. + * @param newVer New version. + */ + public void updateNewVersion(long pageAddr, int dataOff, MvccVersion newVer, byte newTxState) { + long addr = pageAddr + dataOff; + + updateNewVersion(addr, newVer.coordinatorVersion(), newVer.counter(), + (newVer.operationCounter() & ~MVCC_HINTS_MASK) | ((int)newTxState << MVCC_HINTS_BIT_OFF)); + } + + /** + * @param pageAddr Page address. + * @param itemId Item ID. + * @param pageSize Page size. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + * @param mvccOpCntr Operation counter. + */ + public void updateNewVersion(long pageAddr, int itemId, int pageSize, long mvccCrd, long mvccCntr, int mvccOpCntr) { + int dataOff = getDataOffset(pageAddr, itemId, pageSize); + + long addr = pageAddr + dataOff + (isFragmented(pageAddr, dataOff) ? 10 : 2); + + updateNewVersion(addr, mvccCrd, mvccCntr, mvccOpCntr); + } + + /** + * @param pageAddr Page address. + * @param itemId Item ID. + * @param pageSize Page size. + * @param txState Tx state hint. + */ + public void updateTxState(long pageAddr, int itemId, int pageSize, byte txState) { + int dataOff = getDataOffset(pageAddr, itemId, pageSize); + + long addr = pageAddr + dataOff + (isFragmented(pageAddr, dataOff) ? 10 : 2); + + int opCntr = mvccOperationCounter(addr, 0); + + mvccOperationCounter(addr, 0, ((int)txState << MVCC_HINTS_BIT_OFF) | (opCntr & ~MVCC_HINTS_MASK)); + } + + /** + * @param pageAddr Page address. + * @param itemId Item ID. + * @param pageSize Page size. + * @param txState Tx state hint. + */ + public void updateNewTxState(long pageAddr, int itemId, int pageSize, byte txState) { + int dataOff = getDataOffset(pageAddr, itemId, pageSize); + + long addr = pageAddr + dataOff + (isFragmented(pageAddr, dataOff) ? 10 : 2); + + int opCntr = newMvccOperationCounter(addr, 0); + + newMvccOperationCounter(addr, 0, ((int)txState << MVCC_HINTS_BIT_OFF) | (opCntr & ~MVCC_HINTS_MASK)); + } + + /** + * Marks row removed. + * + * @param addr Address. + * @param mvccCrd Mvcc coordinator. + * @param mvccCntr Mvcc counter. + */ + private void updateNewVersion(long addr, long mvccCrd, long mvccCntr, int mvccOpCntr) { + // Skip xid_min. + addr += 20; + + PageUtils.putLong(addr, 0, mvccCrd); + PageUtils.putLong(addr, 8, mvccCntr); + PageUtils.putInt(addr, 16, mvccOpCntr); + } + + /** + * Returns MVCC coordinator number. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @return MVCC coordinator number. + */ + public long mvccCoordinator(long pageAddr, int dataOff) { + long addr = pageAddr + dataOff; + + return PageUtils.getLong(addr, 0); + } + + /** + * Returns MVCC counter value. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @return MVCC counter value. + */ + public long mvccCounter(long pageAddr, int dataOff) { + long addr = pageAddr + dataOff; + + return PageUtils.getLong(addr, 8); + } + + /** + * Returns MVCC operation counter value. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @return MVCC counter value. + */ + public int mvccOperationCounter(long pageAddr, int dataOff) { + long addr = pageAddr + dataOff; + + return PageUtils.getInt(addr, 16); + } + + /** + * Sets MVCC operation counter value. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @param opCntr MVCC counter value. + */ + public void mvccOperationCounter(long pageAddr, int dataOff, int opCntr) { + long addr = pageAddr + dataOff; + + PageUtils.putInt(addr, 16, opCntr); + } + + /** + * Returns new MVCC coordinator number. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @return New MVCC coordinator number. + */ + public long newMvccCoordinator(long pageAddr, int dataOff) { + long addr = pageAddr + dataOff; + + // Skip xid_min. + addr += 20; + + return PageUtils.getLong(addr, 0); + } + + /** + * Returns new MVCC counter value. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @return New MVCC counter value. + */ + public long newMvccCounter(long pageAddr, int dataOff) { + long addr = pageAddr + dataOff; + + // Skip xid_min. + addr += 20; + + return PageUtils.getLong(addr, 8); + } + + /** + * Returns MVCC operation counter value. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @return MVCC counter value. + */ + public int newMvccOperationCounter(long pageAddr, int dataOff) { + long addr = pageAddr + dataOff; + + // Skip xid_min. + addr += 20; + + return PageUtils.getInt(addr, 16); + } + + /** + * Sets MVCC new operation counter value. + * + * @param pageAddr Page address. + * @param dataOff Data offset. + * @param opCntr MVCC operation counter value. + */ + public void newMvccOperationCounter(long pageAddr, int dataOff, int opCntr) { + long addr = pageAddr + dataOff; + + // Skip xid_min. + addr += 20; + + PageUtils.putInt(addr, 16, opCntr); + } + /** * @param buf Byte buffer. * @param ver Version. @@ -181,7 +495,6 @@ private void writeVersionFragment(ByteBuffer buf, GridCacheVersion ver, int rowO */ private void writeExpireTimeFragment(ByteBuffer buf, long expireTime, int rowOff, int len, int prevLen) { int size = 8; - if (size <= len) buf.putLong(expireTime); else { @@ -222,77 +535,35 @@ private void writeCacheIdFragment(ByteBuffer buf, int cacheId, int rowOff, int l } /** - * + * @param buf Byte buffer. + * @param mvccCrd Coordinator version. + * @param mvccCntr Counter. + * @param mvccOpCntr Operation counter. + * @param newMvccCrd New coordinator version. + * @param newMvccCntr New counter version. + * @param newMvccOpCntr New operation counter. + * @param len Length. */ - private enum EntryPart { - /** */ - KEY, - - /** */ - VALUE, - - /** */ - VERSION, - - /** */ - EXPIRE_TIME, - - /** */ - CACHE_ID - } - - /** {@inheritDoc} */ - @Override - protected void writeRowData( - long pageAddr, - int dataOff, - int payloadSize, - CacheDataRow row, - boolean newRow - ) throws IgniteCheckedException { - long addr = pageAddr + dataOff; - - int cacheIdSize = row.cacheId() != 0 ? 4 : 0; - - if (newRow) { - PageUtils.putShort(addr, 0, (short)payloadSize); - addr += 2; - - if (cacheIdSize != 0) { - PageUtils.putInt(addr, 0, row.cacheId()); - - addr += cacheIdSize; - } - - addr += row.key().putValue(addr); - } - else - addr += (2 + cacheIdSize + row.key().valueBytesLength(null)); + private void writeMvccInfoFragment(ByteBuffer buf, long mvccCrd, long mvccCntr, int mvccOpCntr, long newMvccCrd, + long newMvccCntr, int newMvccOpCntr, int len) { + if (mvccCrd == 0) + return; - addr += row.value().putValue(addr); + assert len >= MVCC_INFO_SIZE : "Mvcc info should fit on the one page!"; - CacheVersionIO.write(addr, row.version(), false); - addr += CacheVersionIO.size(row.version(), false); + assert MvccUtils.mvccVersionIsValid(mvccCrd, mvccCntr, mvccOpCntr); - PageUtils.putLong(addr, 0, row.expireTime()); - } + // xid_min. + buf.putLong(mvccCrd); + buf.putLong(mvccCntr); + buf.putInt(mvccOpCntr); - /** {@inheritDoc} */ - @Override - protected void writeRowData( - long pageAddr, - int dataOff, - byte[] payload - ) { - PageUtils.putShort(pageAddr, dataOff, (short)payload.length); - dataOff += 2; - - PageUtils.putBytes(pageAddr, dataOff, payload); - } + assert newMvccCrd == 0 || MvccUtils.mvccVersionIsValid(newMvccCrd, newMvccCntr, newMvccOpCntr); - /** {@inheritDoc} */ - @Override public int getRowSize(CacheDataRow row) throws IgniteCheckedException { - return getRowSize(row, row.cacheId() != 0); + // xid_max. + buf.putLong(newMvccCrd); + buf.putLong(newMvccCntr); + buf.putInt(newMvccOpCntr); } /** {@inheritDoc} */ @@ -303,18 +574,25 @@ protected void writeRowData( } /** - * @param row Row. - * @param withCacheId If {@code true} adds cache ID size. - * @return Entry size on page. - * @throws IgniteCheckedException If failed. + * */ - public static int getRowSize(CacheDataRow row, boolean withCacheId) throws IgniteCheckedException { - KeyCacheObject key = row.key(); - CacheObject val = row.value(); + enum EntryPart { + /** */ + KEY, - int keyLen = key.valueBytesLength(null); - int valLen = val.valueBytesLength(null); + /** */ + VALUE, + + /** */ + VERSION, - return keyLen + valLen + CacheVersionIO.size(row.version(), false) + 8 + (withCacheId ? 4 : 0); + /** */ + EXPIRE_TIME, + + /** */ + CACHE_ID, + + /** */ + MVCC_INFO } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java index d74d34462ba79..9dcad9bdcb2a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/IOVersions.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.cache.persistence.tree.io; +import org.apache.ignite.internal.util.typedef.internal.S; + /** * Registry for IO versions. */ @@ -99,4 +101,9 @@ public V forPage(long pageAddr) { return res; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IOVersions.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java index 4534bb5d63cfa..f167d0cb11cb9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java @@ -25,6 +25,8 @@ import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLogInnerIO; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLogLeafIO; import org.apache.ignite.internal.processors.cache.persistence.IndexStorageImpl; import org.apache.ignite.internal.processors.cache.persistence.freelist.io.PagesListMetaIO; import org.apache.ignite.internal.processors.cache.persistence.freelist.io.PagesListNodeIO; @@ -33,10 +35,14 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener; import org.apache.ignite.internal.processors.cache.tree.CacheIdAwareDataInnerIO; import org.apache.ignite.internal.processors.cache.tree.CacheIdAwareDataLeafIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccCacheIdAwareDataInnerIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccCacheIdAwareDataLeafIO; import org.apache.ignite.internal.processors.cache.tree.CacheIdAwarePendingEntryInnerIO; import org.apache.ignite.internal.processors.cache.tree.CacheIdAwarePendingEntryLeafIO; import org.apache.ignite.internal.processors.cache.tree.DataInnerIO; import org.apache.ignite.internal.processors.cache.tree.DataLeafIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataInnerIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataLeafIO; import org.apache.ignite.internal.processors.cache.tree.PendingEntryInnerIO; import org.apache.ignite.internal.processors.cache.tree.PendingEntryLeafIO; import org.apache.ignite.internal.util.GridStringBuilder; @@ -87,6 +93,12 @@ public abstract class PageIO { /** */ private static IOVersions> h2LeafIOs; + /** */ + private static IOVersions> h2MvccInnerIOs; + + /** */ + private static IOVersions> h2MvccLeafIOs; + /** Maximum payload size. */ public static final short MAX_PAYLOAD_SIZE = 2048; @@ -96,6 +108,12 @@ public abstract class PageIO { /** */ private static List>> h2ExtraLeafIOs = new ArrayList<>(MAX_PAYLOAD_SIZE); + /** */ + private static List>> h2ExtraMvccInnerIOs = new ArrayList<>(MAX_PAYLOAD_SIZE); + + /** */ + private static List>> h2ExtraMvccLeafIOs = new ArrayList<>(MAX_PAYLOAD_SIZE); + /** */ public static final int TYPE_OFF = 0; @@ -129,6 +147,12 @@ public abstract class PageIO { /** */ public static final int COMMON_HEADER_END = RESERVED_3_OFF + 8; // 40=type(2)+ver(2)+crc(4)+pageId(8)+rotatedIdPart(1)+reserved(1+2+4+2*8) + /** */ + public static final int MVCC_HINTS_MASK = 0xC0000000; + + /** */ + public static final int MVCC_HINTS_BIT_OFF = 30; + /* All the page types. */ /** */ @@ -200,18 +224,54 @@ public abstract class PageIO { /** */ public static final short T_DATA_REF_METASTORAGE_LEAF = 23; + /** */ + public static final short T_DATA_REF_MVCC_INNER = 24; + + /** */ + public static final short T_DATA_REF_MVCC_LEAF = 25; + + /** */ + public static final short T_CACHE_ID_DATA_REF_MVCC_INNER = 26; + + /** */ + public static final short T_CACHE_ID_DATA_REF_MVCC_LEAF = 27; + + /** */ + public static final short T_H2_MVCC_REF_LEAF = 28; + + /** */ + public static final short T_H2_MVCC_REF_INNER = 29; + + /** */ + public static final short T_TX_LOG_LEAF = 30; + + /** */ + public static final short T_TX_LOG_INNER = 31; + /** Index for payload == 1. */ - public static final short T_H2_EX_REF_LEAF_START = 10000; + public static final short T_H2_EX_REF_LEAF_START = 10_000; /** */ public static final short T_H2_EX_REF_LEAF_END = T_H2_EX_REF_LEAF_START + MAX_PAYLOAD_SIZE - 1; /** */ - public static final short T_H2_EX_REF_INNER_START = 20000; + public static final short T_H2_EX_REF_INNER_START = 20_000; /** */ public static final short T_H2_EX_REF_INNER_END = T_H2_EX_REF_INNER_START + MAX_PAYLOAD_SIZE - 1; + /** */ + public static final short T_H2_EX_REF_MVCC_LEAF_START = 23_000; + + /** */ + public static final short T_H2_EX_REF_MVCC_LEAF_END = T_H2_EX_REF_MVCC_LEAF_START + MAX_PAYLOAD_SIZE - 1; + + /** */ + public static final short T_H2_EX_REF_MVCC_INNER_START = 26_000; + + /** */ + public static final short T_H2_EX_REF_MVCC_INNER_END = T_H2_EX_REF_MVCC_INNER_START + MAX_PAYLOAD_SIZE - 1; + /** */ private final int ver; @@ -363,13 +423,19 @@ public static void setCrc(ByteBuffer buf, int crc) { * * @param innerIOs Inner IO versions. * @param leafIOs Leaf IO versions. + * @param mvccInnerIOs Inner IO versions with mvcc enabled. + * @param mvccLeafIOs Leaf IO versions with mvcc enabled. */ public static void registerH2( IOVersions> innerIOs, - IOVersions> leafIOs + IOVersions> leafIOs, + IOVersions> mvccInnerIOs, + IOVersions> mvccLeafIOs ) { h2InnerIOs = innerIOs; h2LeafIOs = leafIOs; + h2MvccInnerIOs = mvccInnerIOs; + h2MvccLeafIOs = mvccLeafIOs; } /** @@ -377,8 +443,10 @@ public static void registerH2( * * @param innerExtIOs Extra versions. */ - public static void registerH2ExtraInner(IOVersions> innerExtIOs) { - h2ExtraInnerIOs.add(innerExtIOs); + public static void registerH2ExtraInner(IOVersions> innerExtIOs, boolean mvcc) { + List>> ios = mvcc ? h2ExtraMvccInnerIOs : h2ExtraInnerIOs; + + ios.add(innerExtIOs); } /** @@ -386,24 +454,30 @@ public static void registerH2ExtraInner(IOVersions> in * * @param leafExtIOs Extra versions. */ - public static void registerH2ExtraLeaf(IOVersions> leafExtIOs) { - h2ExtraLeafIOs.add(leafExtIOs); + public static void registerH2ExtraLeaf(IOVersions> leafExtIOs, boolean mvcc) { + List>> ios = mvcc ? h2ExtraMvccLeafIOs : h2ExtraLeafIOs; + + ios.add(leafExtIOs); } /** * @param idx Index. * @return IOVersions for given idx. */ - public static IOVersions> getInnerVersions(int idx) { - return h2ExtraInnerIOs.get(idx); + public static IOVersions> getInnerVersions(int idx, boolean mvcc) { + List>> ios = mvcc ? h2ExtraMvccInnerIOs : h2ExtraInnerIOs; + + return ios.get(idx); } /** * @param idx Index. * @return IOVersions for given idx. */ - public static IOVersions> getLeafVersions(int idx) { - return h2ExtraLeafIOs.get(idx); + public static IOVersions> getLeafVersions(int idx, boolean mvcc) { + List>> ios = mvcc ? h2ExtraMvccLeafIOs : h2ExtraLeafIOs; + + return ios.get(idx); } /** @@ -525,13 +599,18 @@ public static > Q getBPlusIO(long pageAddr) throws IgniteCh */ @SuppressWarnings("unchecked") public static > Q getBPlusIO(int type, int ver) throws IgniteCheckedException { - if (type >= T_H2_EX_REF_LEAF_START && type <= T_H2_EX_REF_LEAF_END) return (Q)h2ExtraLeafIOs.get(type - T_H2_EX_REF_LEAF_START).forVersion(ver); if (type >= T_H2_EX_REF_INNER_START && type <= T_H2_EX_REF_INNER_END) return (Q)h2ExtraInnerIOs.get(type - T_H2_EX_REF_INNER_START).forVersion(ver); + if (type >= T_H2_EX_REF_MVCC_LEAF_START && type <= T_H2_EX_REF_MVCC_LEAF_END) + return (Q)h2ExtraMvccLeafIOs.get(type - T_H2_EX_REF_MVCC_LEAF_START).forVersion(ver); + + if (type >= T_H2_EX_REF_MVCC_INNER_START && type <= T_H2_EX_REF_MVCC_INNER_END) + return (Q)h2ExtraMvccInnerIOs.get(type - T_H2_EX_REF_MVCC_INNER_START).forVersion(ver); + switch (type) { case T_H2_REF_INNER: if (h2InnerIOs == null) @@ -545,6 +624,24 @@ public static > Q getBPlusIO(int type, int ver) throws Igni return (Q)h2LeafIOs.forVersion(ver); + case T_H2_MVCC_REF_INNER: + if (h2MvccInnerIOs == null) + break; + + return (Q)h2MvccInnerIOs.forVersion(ver); + + case T_H2_MVCC_REF_LEAF: + if (h2MvccLeafIOs == null) + break; + + return (Q)h2MvccLeafIOs.forVersion(ver); + + case T_TX_LOG_INNER: + return (Q)TxLogInnerIO.VERSIONS.forVersion(ver); + + case T_TX_LOG_LEAF: + return (Q)TxLogLeafIO.VERSIONS.forVersion(ver); + case T_DATA_REF_INNER: return (Q)DataInnerIO.VERSIONS.forVersion(ver); @@ -557,6 +654,18 @@ public static > Q getBPlusIO(int type, int ver) throws Igni case T_CACHE_ID_AWARE_DATA_REF_LEAF: return (Q)CacheIdAwareDataLeafIO.VERSIONS.forVersion(ver); + case T_CACHE_ID_DATA_REF_MVCC_INNER: + return (Q) MvccCacheIdAwareDataInnerIO.VERSIONS.forVersion(ver); + + case T_CACHE_ID_DATA_REF_MVCC_LEAF: + return (Q) MvccCacheIdAwareDataLeafIO.VERSIONS.forVersion(ver); + + case T_DATA_REF_MVCC_INNER: + return (Q)MvccDataInnerIO.VERSIONS.forVersion(ver); + + case T_DATA_REF_MVCC_LEAF: + return (Q)MvccDataLeafIO.VERSIONS.forVersion(ver); + case T_METASTORE_INNER: return (Q)IndexStorageImpl.MetaStoreInnerIO.VERSIONS.forVersion(ver); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/SimpleDataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/SimpleDataPageIO.java index f02e3a0366882..14489b7cd0fb2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/SimpleDataPageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/SimpleDataPageIO.java @@ -112,12 +112,6 @@ public static byte[] readPayload(long link) { return PageUtils.getBytes(link, 4, size); } - /** {@inheritDoc} */ - @Override public int getRowSize(MetastorageDataRow row) throws IgniteCheckedException { - return 4 + row.value().length; - } - - /** {@inheritDoc} */ @Override protected void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException { sb.a("SimpleDataPageIO [\n"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java index b9ab76a3c7b05..f160549b33204 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java @@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; import org.apache.ignite.internal.processors.cache.GridCacheProcessor; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; +import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; @@ -466,6 +467,11 @@ protected IgniteConfiguration prepareIgniteConfiguration() { return null; } + /** {@inheritDoc} */ + @Override public MvccProcessor coordinators() { + return null; + } + /** {@inheritDoc} */ @Override public boolean invalid() { return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java index 1807d1d22f212..65f0aae83b2a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java @@ -35,6 +35,9 @@ public final class RecordTypes { DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_INSERT_FRAGMENT_RECORD); DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_REMOVE_RECORD); DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_SET_FREE_LIST_PAGE); + DELTA_TYPE_SET.add(WALRecord.RecordType.MVCC_DATA_PAGE_MARK_UPDATED_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.MVCC_DATA_PAGE_TX_STATE_HINT_UPDATED_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.MVCC_DATA_PAGE_NEW_TX_STATE_HINT_UPDATED_RECORD); DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_META_PAGE_INIT_ROOT); DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_META_PAGE_ADD_ROOT); DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_META_PAGE_CUT_ROOT); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java index ad060906767dc..aa89c5a35cb9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java @@ -41,6 +41,8 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertFragmentRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertRecord; +import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccUpdateNewTxStateHintRecord; +import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccUpdateTxStateHintRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageRemoveRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageSetFreeListPageRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageUpdateRecord; @@ -61,6 +63,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastSuccessfulSnapshotId; import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateNextSnapshotId; import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdatePartitionDataRecord; +import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccMarkUpdatedRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.NewRootInitRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.PageListMetaResetCountRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.PagesListAddPageRecord; @@ -194,6 +197,15 @@ assert record instanceof PageSnapshot; case DATA_PAGE_SET_FREE_LIST_PAGE: return 4 + 8 + 8; + case MVCC_DATA_PAGE_MARK_UPDATED_RECORD: + return 4 + 8 + 4 + 8 + 8 + 4; + + case MVCC_DATA_PAGE_TX_STATE_HINT_UPDATED_RECORD: + return 4 + 8 + 4 + 1; + + case MVCC_DATA_PAGE_NEW_TX_STATE_HINT_UPDATED_RECORD: + return 4 + 8 + 4 + 1; + case INIT_NEW_PAGE_RECORD: return 4 + 8 + 2 + 2 + 8; @@ -504,6 +516,41 @@ assert record instanceof PageSnapshot; break; + case MVCC_DATA_PAGE_MARK_UPDATED_RECORD: + cacheId = in.readInt(); + pageId = in.readLong(); + + itemId = in.readInt(); + long newMvccCrd = in.readLong(); + long newMvccCntr = in.readLong(); + int newMvccOpCntr = in.readInt(); + + res = new DataPageMvccMarkUpdatedRecord(cacheId, pageId, itemId, newMvccCrd, newMvccCntr, newMvccOpCntr); + + break; + + case MVCC_DATA_PAGE_TX_STATE_HINT_UPDATED_RECORD: + cacheId = in.readInt(); + pageId = in.readLong(); + + itemId = in.readInt(); + byte txState = in.readByte(); + + res = new DataPageMvccUpdateTxStateHintRecord(cacheId, pageId, itemId, txState); + + break; + + case MVCC_DATA_PAGE_NEW_TX_STATE_HINT_UPDATED_RECORD: + cacheId = in.readInt(); + pageId = in.readLong(); + + itemId = in.readInt(); + byte newTxState = in.readByte(); + + res = new DataPageMvccUpdateNewTxStateHintRecord(cacheId, pageId, itemId, newTxState); + + break; + case INIT_NEW_PAGE_RECORD: cacheId = in.readInt(); pageId = in.readLong(); @@ -1035,6 +1082,41 @@ assert record instanceof PageSnapshot; break; + case MVCC_DATA_PAGE_MARK_UPDATED_RECORD: + DataPageMvccMarkUpdatedRecord rmvRec = (DataPageMvccMarkUpdatedRecord)rec; + + buf.putInt(rmvRec.groupId()); + buf.putLong(rmvRec.pageId()); + + buf.putInt(rmvRec.itemId()); + buf.putLong(rmvRec.newMvccCrd()); + buf.putLong(rmvRec.newMvccCntr()); + buf.putInt(rmvRec.newMvccOpCntr()); + + break; + + case MVCC_DATA_PAGE_TX_STATE_HINT_UPDATED_RECORD: + DataPageMvccUpdateTxStateHintRecord txStRec = (DataPageMvccUpdateTxStateHintRecord)rec; + + buf.putInt(txStRec.groupId()); + buf.putLong(txStRec.pageId()); + + buf.putInt(txStRec.itemId()); + buf.put(txStRec.txState()); + + break; + + case MVCC_DATA_PAGE_NEW_TX_STATE_HINT_UPDATED_RECORD: + DataPageMvccUpdateNewTxStateHintRecord newTxStRec = (DataPageMvccUpdateNewTxStateHintRecord)rec; + + buf.putInt(newTxStRec.groupId()); + buf.putLong(newTxStRec.pageId()); + + buf.putInt(newTxStRec.itemId()); + buf.put(newTxStRec.txState()); + + break; + case INIT_NEW_PAGE_RECORD: InitNewPageRecord inpRec = (InitNewPageRecord)rec; 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 aac1659c0fd24..c5f64c902ede6 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 @@ -35,6 +35,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.util.GridBoundedConcurrentOrderedSet; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; @@ -195,6 +196,9 @@ protected void removeQueryFuture(long reqId) { */ @SuppressWarnings("unchecked") @Override void processQueryRequest(UUID sndId, GridCacheQueryRequest req) { + assert req.mvccSnapshot() != null || !cctx.mvccEnabled() || req.cancel() || + (req.type() == null && !req.fields()) : req; // Last assertion means next page request. + if (req.cancel()) { cancelIds.add(new CancelMessageId(req.id(), sndId)); @@ -277,7 +281,8 @@ protected void removeQueryFuture(long reqId) { req.includeMetaData(), req.keepBinary(), req.subjectId(), - req.taskHash() + req.taskHash(), + req.mvccSnapshot() ); return new GridCacheQueryInfo( @@ -531,6 +536,8 @@ else if (!cancelled.contains(res.requestId())) String clsName = qry.query().queryClassName(); + MvccSnapshot mvccSnapshot = qry.query().mvccSnapshot(); + final GridCacheQueryRequest req = new GridCacheQueryRequest( cctx.cacheId(), reqId, @@ -551,6 +558,7 @@ else if (!cancelled.contains(res.requestId())) qry.query().subjectId(), qry.query().taskHash(), queryTopologyVersion(), + mvccSnapshot, // Force deployment anyway if scan query is used. cctx.deploymentEnabled() || (qry.query().scanFilter() != null && cctx.gridDeploy().enabled())); @@ -581,6 +589,7 @@ else if (!cancelled.contains(res.requestId())) Collection nodes) throws IgniteCheckedException { assert !cctx.isLocal() : cctx.name(); assert qry.type() == GridCacheQueryType.SCAN: qry; + assert qry.mvccSnapshot() != null || !cctx.mvccEnabled(); GridCloseableIterator locIter0 = null; @@ -606,7 +615,7 @@ else if (!cancelled.contains(res.requestId())) final GridCacheQueryBean bean = new GridCacheQueryBean(qry, null, qry.transform(), null); - final CacheQueryFuture fut = (CacheQueryFuture)queryDistributed(bean, nodes); + final CacheQueryFuture fut = queryDistributed(bean, nodes); return new GridCloseableIteratorAdapter() { /** */ @@ -749,6 +758,7 @@ private Object convert(Object obj) { qry.query().subjectId(), qry.query().taskHash(), queryTopologyVersion(), + null, cctx.deploymentEnabled()); addQueryFuture(req.id(), fut); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java index 51fdd5866ea82..f21a22f8214fc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.Deque; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedList; import java.util.Map; import java.util.NoSuchElementException; @@ -29,6 +30,7 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.query.Query; @@ -43,6 +45,9 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.GridEmptyCloseableIterator; @@ -60,6 +65,7 @@ import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteReducer; import org.apache.ignite.plugin.security.SecurityPermission; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheMode.LOCAL; @@ -127,6 +133,9 @@ public class GridCacheQueryAdapter implements CacheQuery { /** */ private int taskHash; + /** */ + private MvccSnapshot mvccSnapshot; + /** * @param cctx Context. * @param type Query type. @@ -213,6 +222,7 @@ public GridCacheQueryAdapter(GridCacheContext cctx, * @param keepBinary Keep binary flag. * @param subjId Security subject ID. * @param taskHash Task hash. + * @param mvccSnapshot Mvcc version. */ public GridCacheQueryAdapter(GridCacheContext cctx, GridCacheQueryType type, @@ -229,7 +239,8 @@ public GridCacheQueryAdapter(GridCacheContext cctx, boolean incMeta, boolean keepBinary, UUID subjId, - int taskHash) { + int taskHash, + MvccSnapshot mvccSnapshot) { this.cctx = cctx; this.type = type; this.log = log; @@ -246,6 +257,14 @@ public GridCacheQueryAdapter(GridCacheContext cctx, this.keepBinary = keepBinary; this.subjId = subjId; this.taskHash = taskHash; + this.mvccSnapshot = mvccSnapshot; + } + + /** + * @return MVCC snapshot. + */ + @Nullable MvccSnapshot mvccSnapshot() { + return mvccSnapshot; } /** @@ -400,7 +419,7 @@ public ClusterGroup projection() { */ @SuppressWarnings("unchecked") @Nullable public IgniteClosure, Object> transform() { - return (IgniteClosure, Object>) transform; + return (IgniteClosure, Object>)transform; } /** @@ -519,15 +538,23 @@ private CacheQueryFuture execute0(@Nullable IgniteReducer rmtReduce final GridCacheQueryManager qryMgr = cctx.queries(); + MvccQueryTracker mvccTracker = null; + + if (cctx.mvccEnabled() && mvccSnapshot == null) + mvccSnapshot = (mvccTracker = MvccUtils.mvccTracker(cctx, false)).snapshot(); + boolean loc = nodes.size() == 1 && F.first(nodes).id().equals(cctx.localNodeId()); - if (loc) - return qryMgr.scanQueryLocal(this, true); + GridCloseableIterator it; - if (part != null) - return new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx); + if (loc) + it = qryMgr.scanQueryLocal(this, true); + else if (part != null) + it = new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx); else - return qryMgr.scanQueryDistributed(this, nodes); + it = qryMgr.scanQueryDistributed(this, nodes); + + return mvccTracker != null ? new MvccTrackingIterator(it, mvccTracker) : it; } /** @@ -781,7 +808,7 @@ private void init() { * @return Cache entry */ private Object convert(Object obj) { - if(qry.transform() != null) + if (qry.transform() != null) return obj; Map.Entry e = (Map.Entry)obj; @@ -853,4 +880,93 @@ else if (e.hasCause(ClusterGroupEmptyCheckedException.class)) { t.get2().cancel(); } } + + /** + * Wrapper for an MVCC-related iterators. + */ + private static class MvccTrackingIterator implements GridCloseableIterator { + /** Serial version uid. */ + private static final long serialVersionUID = -1905248502802333832L; + /** Underlying iterator. */ + private final GridCloseableIterator it; + + /** Query MVCC tracker. */ + private final MvccQueryTracker mvccTracker; + + /** + * Constructor. + * + * @param it Underlying iterator. + * @param mvccTracker Query MVCC tracker. + */ + MvccTrackingIterator(GridCloseableIterator it, MvccQueryTracker mvccTracker) { + assert it != null && mvccTracker != null; + + this.it = it; + this.mvccTracker = mvccTracker; + } + + /** {@inheritDoc} */ + @Override public void close() throws IgniteCheckedException { + if (isClosed()) + return; + + try { + it.close(); + } + finally { + mvccTracker.onDone(); + } + } + + /** {@inheritDoc} */ + @Override public boolean isClosed() { + return it.isClosed(); + } + + /** {@inheritDoc} */ + @Override public boolean hasNext() { + boolean hasNext = it.hasNext(); + + if (!hasNext) + try { + close(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + + return hasNext; + } + + /** {@inheritDoc} */ + @Override public boolean hasNextX() throws IgniteCheckedException { + boolean hasNext = it.hasNext(); + + if (!hasNext) + close(); + + return hasNext; + } + + /** {@inheritDoc} */ + @Override public Object nextX() throws IgniteCheckedException { + return it.nextX(); + } + + /** {@inheritDoc} */ + @Override public void removeX() throws IgniteCheckedException { + it.removeX(); + } + + /** {@inheritDoc} */ + @NotNull @Override public Iterator iterator() { + return this; + } + + /** {@inheritDoc} */ + @Override public Object next() { + return it.next(); + } + } } 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 37bdb80c3ad3e..c209602c62e67 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 @@ -380,8 +380,8 @@ private void invalidateResultCache() { * @param prevRowAvailable Whether previous row is available. * @throws IgniteCheckedException In case of error. */ - public void store(CacheDataRow newRow, @Nullable CacheDataRow prevRow, boolean prevRowAvailable) - throws IgniteCheckedException { + public void store(CacheDataRow newRow, @Nullable CacheDataRow prevRow, + boolean prevRowAvailable) throws IgniteCheckedException { assert enabled(); assert newRow != null && newRow.value() != null && newRow.link() != 0 : newRow; @@ -414,7 +414,8 @@ public void store(CacheDataRow newRow, @Nullable CacheDataRow prevRow, boolean p * @param prevRow Previous row. * @throws IgniteCheckedException Thrown in case of any errors. */ - public void remove(KeyCacheObject key, @Nullable CacheDataRow prevRow) throws IgniteCheckedException { + public void remove(KeyCacheObject key, @Nullable CacheDataRow prevRow) + throws IgniteCheckedException { if (!QueryUtils.isEnabled(cctx.config())) return; // No-op. @@ -803,6 +804,8 @@ private GridCloseableIterator> setIterator(GridCacheQueryAda private GridCloseableIterator scanIterator(final GridCacheQueryAdapter qry, IgniteClosure transformer, boolean locNode) throws IgniteCheckedException { + assert !cctx.mvccEnabled() || qry.mvccSnapshot() != null; + final IgniteBiPredicate keyValFilter = qry.scanFilter(); final InternalScanFilter intFilter = keyValFilter != null ? new InternalScanFilter<>(keyValFilter) : null; @@ -843,13 +846,12 @@ private GridCloseableIterator scanIterator(final GridCacheQueryAdapter qry, I locPart = locPart0; - it = cctx.offheap().cachePartitionIterator(cctx.cacheId(), part); + it = cctx.offheap().cachePartitionIterator(cctx.cacheId(), part, qry.mvccSnapshot()); } else { locPart = null; - // TODO shouldn't we reserve all involved partitions? - it = cctx.offheap().cacheIterator(cctx.cacheId(), true, backups, topVer); + it = cctx.offheap().cacheIterator(cctx.cacheId(), true, backups, topVer, qry.mvccSnapshot()); } return new ScanQueryIterator(it, qry, topVer, locPart, keyValFilter, transformer, locNode, cctx, log); @@ -2923,11 +2925,13 @@ private static final class ScanQueryIterator extends GridCloseableIterator boolean locNode, GridCacheContext cctx, IgniteLogger log) { + this.it = it; this.topVer = topVer; this.locPart = locPart; this.intScanFilter = scanFilter != null ? new InternalScanFilter<>(scanFilter) : null; this.cctx = cctx; + this.log = log; this.locNode = locNode; @@ -2938,7 +2942,7 @@ private static final class ScanQueryIterator extends GridCloseableIterator readEvt = cctx.events().isRecordable(EVT_CACHE_QUERY_OBJECT_READ) && cctx.gridEvents().hasListener(EVT_CACHE_QUERY_OBJECT_READ); - if(readEvt){ + if (readEvt){ taskName = cctx.kernalContext().task().resolveTaskName(qry.taskHash()); subjId = qry.subjectId(); } 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 9dc7817889806..ebbca351aae06 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 @@ -26,8 +26,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheDeployable; import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; -import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -129,6 +129,9 @@ public class GridCacheQueryRequest extends GridCacheIdMessage implements GridCac /** */ private AffinityTopologyVersion topVer; + /** */ + private MvccSnapshot mvccSnapshot; + /** * Required by {@link Externalizable} */ @@ -223,6 +226,7 @@ public GridCacheQueryRequest( * @param subjId Subject ID. * @param taskHash Task name hash code. * @param topVer Topology version. + * @param mvccSnapshot Mvcc snapshot. * @param addDepInfo Deployment info flag. */ public GridCacheQueryRequest( @@ -245,6 +249,7 @@ public GridCacheQueryRequest( UUID subjId, int taskHash, AffinityTopologyVersion topVer, + MvccSnapshot mvccSnapshot, boolean addDepInfo ) { assert type != null || fields; @@ -270,9 +275,17 @@ public GridCacheQueryRequest( this.subjId = subjId; this.taskHash = taskHash; this.topVer = topVer; + this.mvccSnapshot = mvccSnapshot; this.addDepInfo = addDepInfo; } + /** + * @return Mvcc version. + */ + @Nullable MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + /** {@inheritDoc} */ @Override public AffinityTopologyVersion topologyVersion() { return topVer != null ? topVer : AffinityTopologyVersion.NONE; @@ -573,48 +586,54 @@ public int taskHash() { writer.incrementState(); case 15: - if (!writer.writeInt("pageSize", pageSize)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 16: - if (!writer.writeInt("part", part)) + if (!writer.writeInt("pageSize", pageSize)) return false; writer.incrementState(); case 17: - if (!writer.writeByteArray("rdcBytes", rdcBytes)) + if (!writer.writeInt("part", part)) return false; writer.incrementState(); case 18: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeByteArray("rdcBytes", rdcBytes)) return false; writer.incrementState(); case 19: - if (!writer.writeInt("taskHash", taskHash)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 20: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeInt("taskHash", taskHash)) return false; writer.incrementState(); case 21: - if (!writer.writeByteArray("transBytes", transBytes)) + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); case 22: + if (!writer.writeByteArray("transBytes", transBytes)) + return false; + + writer.incrementState(); + + case 23: if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1)) return false; @@ -733,7 +752,7 @@ public int taskHash() { reader.incrementState(); case 15: - pageSize = reader.readInt("pageSize"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -741,7 +760,7 @@ public int taskHash() { reader.incrementState(); case 16: - part = reader.readInt("part"); + pageSize = reader.readInt("pageSize"); if (!reader.isLastRead()) return false; @@ -749,7 +768,7 @@ public int taskHash() { reader.incrementState(); case 17: - rdcBytes = reader.readByteArray("rdcBytes"); + part = reader.readInt("part"); if (!reader.isLastRead()) return false; @@ -757,7 +776,7 @@ public int taskHash() { reader.incrementState(); case 18: - subjId = reader.readUuid("subjId"); + rdcBytes = reader.readByteArray("rdcBytes"); if (!reader.isLastRead()) return false; @@ -765,7 +784,7 @@ public int taskHash() { reader.incrementState(); case 19: - taskHash = reader.readInt("taskHash"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -773,7 +792,7 @@ public int taskHash() { reader.incrementState(); case 20: - topVer = reader.readMessage("topVer"); + taskHash = reader.readInt("taskHash"); if (!reader.isLastRead()) return false; @@ -781,7 +800,7 @@ public int taskHash() { reader.incrementState(); case 21: - transBytes = reader.readByteArray("transBytes"); + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -789,6 +808,14 @@ public int taskHash() { reader.incrementState(); case 22: + transBytes = reader.readByteArray("transBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 23: byte typeOrd; typeOrd = reader.readByte("type"); @@ -812,7 +839,7 @@ public int taskHash() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 23; + return 24; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java index 2547e1b8c313d..5dab5fd0d4b98 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java @@ -106,6 +106,20 @@ public final class IgniteQueryErrorCode { /** Attempt to INSERT, UPDATE or MERGE value that exceed maximum column length. */ public final static int TOO_LONG_VALUE = 4008; + /* 5xxx - transactions related runtime errors. */ + + /** Transaction is already open. */ + public final static int TRANSACTION_EXISTS = 5001; + + /** MVCC disabled. */ + public final static int MVCC_DISABLED = 5002; + + /** Transaction type mismatch (SQL/non SQL). */ + public final static int TRANSACTION_TYPE_MISMATCH = 5003; + + /** Transaction is already completed. */ + public final static int TRANSACTION_COMPLETED = 5004; + /** */ private IgniteQueryErrorCode() { // No-op. @@ -159,6 +173,12 @@ public static String codeToSqlState(int statusCode) { case KEY_UPDATE: return SqlStateCode.PARSING_EXCEPTION; + case MVCC_DISABLED: + case TRANSACTION_EXISTS: + case TRANSACTION_TYPE_MISMATCH: + case TRANSACTION_COMPLETED: + return SqlStateCode.TRANSACTION_STATE_EXCEPTION; + default: return SqlStateCode.INTERNAL_ERROR; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java index ff10e3d63da9a..676e61cbb077c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/SqlFieldsQueryEx.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.processors.query.NestedTxMode; import org.apache.ignite.internal.util.typedef.F; /** @@ -36,6 +37,12 @@ public final class SqlFieldsQueryEx extends SqlFieldsQuery { /** Whether server side DML should be enabled. */ private boolean skipReducerOnUpdate; + /** Auto commit flag. */ + private boolean autoCommit = true; + + /** Nested transactions handling mode. */ + private NestedTxMode nestedTxMode = NestedTxMode.DEFAULT; + /** Batched arguments list. */ private List batchedArgs; @@ -57,6 +64,8 @@ private SqlFieldsQueryEx(SqlFieldsQueryEx qry) { this.isQry = qry.isQry; this.skipReducerOnUpdate = qry.skipReducerOnUpdate; + this.autoCommit = qry.autoCommit; + this.nestedTxMode = qry.nestedTxMode; this.batchedArgs = qry.batchedArgs; } @@ -159,6 +168,36 @@ public boolean isSkipReducerOnUpdate() { return skipReducerOnUpdate; } + /** + * @return Nested transactions handling mode - behavior when the user attempts to open a transaction in scope of + * another transaction. + */ + public NestedTxMode getNestedTxMode() { + return nestedTxMode; + } + + /** + * @param nestedTxMode Nested transactions handling mode - behavior when the user attempts to open a transaction + * in scope of another transaction. + */ + public void setNestedTxMode(NestedTxMode nestedTxMode) { + this.nestedTxMode = nestedTxMode; + } + + /** + * @return Auto commit flag. + */ + public boolean isAutoCommit() { + return autoCommit; + } + + /** + * @param autoCommit Auto commit flag. + */ + public void setAutoCommit(boolean autoCommit) { + this.autoCommit = autoCommit; + } + /** {@inheritDoc} */ @Override public SqlFieldsQuery copy() { return new SqlFieldsQueryEx(this); 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 4005dd8fbbbe3..996e7f4c52ed5 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 @@ -29,6 +29,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -76,7 +77,6 @@ import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.resources.LoggerResource; import org.jetbrains.annotations.Nullable; -import java.util.concurrent.ConcurrentHashMap; import static javax.cache.event.EventType.CREATED; import static javax.cache.event.EventType.EXPIRED; @@ -481,6 +481,11 @@ public UUID executeQuery(@Nullable final CacheEntryUpdatedListener locLsnr, final boolean keepBinary, final boolean includeExpired) throws IgniteCheckedException { + //TODO IGNITE-7953 + if (!cctx.atomic() && cctx.kernalContext().config().isMvccEnabled()) + throw new UnsupportedOperationException("Continuous queries are not supported for transactional caches " + + "when MVCC is enabled."); + IgniteOutClosure clsr; if (rmtTransFactory != null) { @@ -741,7 +746,8 @@ private UUID executeQuery0(CacheEntryUpdatedListener locLsnr, final Iterator it = cctx.offheap().cacheIterator(cctx.cacheId(), true, true, - AffinityTopologyVersion.NONE); + AffinityTopologyVersion.NONE, + null); locLsnr.onUpdated(new Iterable() { @Override public Iterator iterator() { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java index 9e06d9d017bae..4acf078bc8038 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheFilterFailedException; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.lang.GridTuple; @@ -261,6 +262,11 @@ public enum FinalizationStatus { */ public boolean activeCachesDeploymentEnabled(); + /** + * @param depEnabled Flag indicating whether deployment is enabled for caches from this transaction or not. + */ + public void activeCachesDeploymentEnabled(boolean depEnabled); + /** * Attempts to set topology version and returns the current value. * If topology version was previously set, then it's value will @@ -634,4 +640,14 @@ public void completedVersions(GridCacheVersion base, * @param e Commit error. */ public void commitError(Throwable e); + + /** + * @param mvccSnapshot Mvcc snapshot. + */ + public void mvccSnapshot(MvccSnapshot mvccSnapshot); + + /** + * @return Mvcc snapshot. + */ + public MvccSnapshot mvccSnapshot(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java index 25ba849b37d1d..11bf219f6f20b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java @@ -31,9 +31,9 @@ import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionException; import org.apache.ignite.transactions.TransactionIsolation; import org.apache.ignite.transactions.TransactionMetrics; -import org.apache.ignite.transactions.TransactionException; import org.jetbrains.annotations.Nullable; /** @@ -48,6 +48,7 @@ public class IgniteTransactionsImpl implements IgniteTransactionsEx { /** * @param cctx Cache shared context. + * @param lb Label. */ public IgniteTransactionsImpl(GridCacheSharedContext cctx, @Nullable String lb) { this.cctx = cctx; @@ -175,6 +176,7 @@ private GridNearTxLocal txStart0( isolation, timeout, true, + null, txSize, lb ); 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 bdd0c53d6c727..ee5a58ee49129 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 @@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.cache.CacheInvokeEntry; import org.apache.ignite.internal.processors.cache.CacheLazyEntry; import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; @@ -62,6 +63,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; import org.apache.ignite.internal.processors.cache.version.GridCacheLazyPlainVersionedEntry; @@ -71,6 +74,7 @@ import org.apache.ignite.internal.processors.cluster.BaselineTopology; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.GridIntIterator; import org.apache.ignite.internal.util.GridSetWrapper; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter; @@ -84,6 +88,7 @@ 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.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.thread.IgniteThread; import org.apache.ignite.transactions.TransactionConcurrency; @@ -263,6 +268,13 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement /** UUID to consistent id mapper. */ protected ConsistentIdMapper consistentIdMapper; + /** Mvcc tx update snapshot. */ + protected volatile MvccSnapshot mvccSnapshot; + + /** Rollback finish future. */ + @GridToStringExclude + private volatile IgniteInternalFuture rollbackFut; + /** * Empty constructor required for {@link Externalizable}. */ @@ -381,6 +393,18 @@ protected IgniteTxAdapter( consistentIdMapper = new ConsistentIdMapper(cctx.discovery()); } + /** + * @return Mvcc info. + */ + @Override @Nullable public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + + /** {@inheritDoc} */ + @Override public void mvccSnapshot(MvccSnapshot mvccSnapshot) { + this.mvccSnapshot = mvccSnapshot; + } + /** * @return Shared cache context. */ @@ -686,6 +710,20 @@ public void needReturnValue(boolean needRetVal) { this.needRetVal = needRetVal; } + /** + * @return Rollback future. + */ + public IgniteInternalFuture rollbackFuture() { + return rollbackFut; + } + + /** + * @param fut Rollback future. + */ + public void rollbackFuture(IgniteInternalFuture fut) { + rollbackFut = fut; + } + /** * Gets remaining allowed transaction time. * @@ -1109,9 +1147,62 @@ protected final boolean state(TransactionState state, boolean timedOut) { if (state != ACTIVE && state != SUSPENDED) seal(); - if (cctx.wal() != null && cctx.tm().logTxRecords() && txNodes != null) { + if (state == PREPARED || state == COMMITTED || state == ROLLED_BACK) { + if (mvccSnapshot != null) { + byte txState; + + switch (state) { + case PREPARED: + txState = TxState.PREPARED; + break; + case ROLLED_BACK: + txState = TxState.ABORTED; + break; + case COMMITTED: + txState = TxState.COMMITTED; + break; + default: + throw new IllegalStateException("Illegal state: " + state); + } + + try { + if (!cctx.localNode().isClient()) { + if (dht() && remote()) + cctx.coordinators().updateState(mvccSnapshot, txState, false); + else if (local()) { + IgniteInternalFuture rollbackFut = rollbackFuture(); + + boolean syncUpdate = txState == TxState.PREPARED || txState == TxState.COMMITTED || + rollbackFut == null || rollbackFut.isDone(); + + if (syncUpdate) + cctx.coordinators().updateState(mvccSnapshot, txState); + else { + // If tx was aborted, we need to wait tx log is updated on all backups. + rollbackFut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + try { + cctx.coordinators().updateState(mvccSnapshot, txState); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to log TxState: " + txState, e); + } + } + }); + } + } + } + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to log TxState: " + txState, e); + + throw new IgniteException("Failed to log TxState: " + txState, e); + } + } + // Log tx state change to WAL. - if (state == PREPARED || state == COMMITTED || state == ROLLED_BACK) { + if (cctx.wal() != null && cctx.tm().logTxRecords() && txNodes != null) { + BaselineTopology baselineTop = cctx.kernalContext().state().clusterState().baselineTopology(); Map> participatingNodes = consistentIdMapper @@ -1569,7 +1660,8 @@ else if (txEntry.hasOldValue()) /*closure name */recordEvt ? F.first(txEntry.entryProcessors()).get1() : null, resolveTaskName(), null, - keepBinary); + keepBinary, + null); // TODO IGNITE-7371 } boolean modified = false; @@ -1756,6 +1848,32 @@ else if (op == DELETE && resVal != null) return F.t(op, ctx); } + /** + * Notify Dr on tx finished. + * + * @param commit {@code True} if commited, {@code False} otherwise. + */ + protected void notifyDrManager(boolean commit) { + if (system() || internal()) + return; + + IgniteTxState txState = txState(); + + if (mvccSnapshot == null || txState.cacheIds().isEmpty()) + return; + + GridIntIterator iter = txState.cacheIds().iterator(); + + while (iter.hasNext()) { + int cacheId = iter.next(); + + GridCacheContext ctx0 = cctx.cacheContext(cacheId); + + if (ctx0.isDrEnabled()) + ctx0.dr().onTxFinished(mvccSnapshot, commit, topologyVersionSnapshot()); + } + } + /** * @param e Transaction entry. * @param primaryOnly Flag to include backups into check or not. @@ -1886,6 +2004,16 @@ protected Object readResolve() throws ObjectStreamException { return xidVer.hashCode(); } + /** + * Adds cache to the list of active caches in transaction. + * + * @param cacheCtx Cache context to add. + * @param recovery Recovery flag. See {@link CacheOperationContext#setRecovery(boolean)}. + * @throws IgniteCheckedException If caches already enlisted in this transaction are not compatible with given + * cache (e.g. they have different stores). + */ + public abstract void addActiveCache(GridCacheContext cacheCtx, boolean recovery) throws IgniteCheckedException; + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(IgniteTxAdapter.class, this, @@ -1959,6 +2087,16 @@ private static class TxShadow implements IgniteInternalTx { this.rollbackOnly = rollbackOnly; } + /** {@inheritDoc} */ + @Override public void mvccSnapshot(MvccSnapshot mvccSnapshot) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public MvccSnapshot mvccSnapshot() { + return null; + } + /** {@inheritDoc} */ @Override public boolean localResult() { return false; @@ -2039,6 +2177,11 @@ private static class TxShadow implements IgniteInternalTx { return false; } + /** {@inheritDoc} */ + @Override public void activeCachesDeploymentEnabled(boolean depEnabled) { + throw new IllegalStateException("Deserialized transaction can only be used as read-only."); + } + /** {@inheritDoc} */ @Nullable @Override public Object addMeta(int key, Object val) { throw new IllegalStateException("Deserialized transaction can only be used as read-only."); 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 2eba3ac8e8723..7541b433ec478 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 @@ -342,8 +342,6 @@ public IgniteInternalFuture prepareNearTxLocal(final return new GridFinishedFuture<>(e); } - assert firstEntry != null : req; - GridDhtTxLocal tx = null; GridCacheVersion mappedVer = ctx.tm().mappedVersion(req.version()); @@ -363,6 +361,8 @@ public IgniteInternalFuture prepareNearTxLocal(final GridDhtPartitionTopology top = null; if (req.firstClientRequest()) { + assert firstEntry != null : req; + assert req.concurrency() == OPTIMISTIC : req; assert nearNode.isClient() : nearNode; @@ -658,6 +658,8 @@ private boolean needRemap(AffinityTopologyVersion expVer, if (expVer.equals(curVer)) return false; + // TODO IGNITE-6754 check mvcc crd for mvcc enabled txs. + for (IgniteTxEntry e : F.concat(false, req.reads(), req.writes())) { GridCacheContext ctx = e.context(); @@ -912,8 +914,11 @@ private IgniteInternalFuture finishDhtLocal(UUID nodeId, else tx = ctx.tm().tx(dhtVer); - if (tx != null) + if (tx != null) { + tx.mvccSnapshot(req.mvccSnapshot()); + req.txState(tx.txState()); + } if (tx == null && locTx != null && !req.commit()) { U.warn(log, "DHT local tx not found for near local tx rollback " + @@ -1378,6 +1383,7 @@ else if (log.isDebugEnabled()) tx.commitVersion(req.commitVersion()); tx.invalidate(req.isInvalidate()); tx.systemInvalidate(req.isSystemInvalidate()); + tx.mvccSnapshot(req.mvccSnapshot()); // Complete remote candidates. tx.doneRemote(req.baseVersion(), null, null, null); @@ -1385,11 +1391,13 @@ else if (log.isDebugEnabled()) tx.setPartitionUpdateCounters( req.partUpdateCounters() != null ? req.partUpdateCounters().array() : null); + tx.updateCountersMap(req.updateCountersMap()); + tx.commitRemoteTx(); } else { tx.doneRemote(req.baseVersion(), null, null, null); - + tx.mvccSnapshot(req.mvccSnapshot()); tx.rollbackRemoteTx(); } } @@ -1424,6 +1432,7 @@ protected void finish( try { tx.commitVersion(req.writeVersion()); tx.invalidate(req.isInvalidate()); + tx.mvccSnapshot(req.mvccSnapshot()); // Complete remote candidates. tx.doneRemote(req.version(), null, null, null); @@ -1608,10 +1617,12 @@ private void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean committe GridDhtTxPrepareRequest req, GridDhtTxPrepareResponse res ) throws IgniteCheckedException { - if (!F.isEmpty(req.writes())) { + if (req.queryUpdate() || !F.isEmpty(req.writes())) { GridDhtTxRemote tx = ctx.tm().tx(req.version()); if (tx == null) { + assert !req.queryUpdate(); + boolean single = req.last() && req.writes().size() == 1; tx = new GridDhtTxRemote( @@ -1718,7 +1729,8 @@ private void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean committe /*transformClo*/null, tx.resolveTaskName(), /*expiryPlc*/null, - /*keepBinary*/true); + /*keepBinary*/true, + null); // TODO IGNITE-7371 if (val == null) val = cacheCtx.toCacheObject(cacheCtx.store().load(null, entry.key())); @@ -1766,7 +1778,7 @@ private void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean committe res.invalidPartitionsByCacheId(tx.invalidPartitions()); - if (tx.empty() && req.last()) { + if (!req.queryUpdate() && tx.empty() && req.last()) { tx.skipCompletedVersions(req.skipCompletedVersion()); tx.rollbackRemoteTx(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java index 10b06d8b97213..4619a80962969 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; +import org.apache.ignite.internal.util.GridIntList; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -57,9 +58,10 @@ public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter { private boolean recovery; /** {@inheritDoc} */ - @Override public void addActiveCache(GridCacheContext ctx, boolean recovery, IgniteTxLocalAdapter tx) + @Override public void addActiveCache(GridCacheContext ctx, boolean recovery, IgniteTxAdapter tx) throws IgniteCheckedException { assert cacheCtx == null : "Cache already set [cur=" + cacheCtx.name() + ", new=" + ctx.name() + ']'; + assert tx.local(); cacheCtx = ctx; this.recovery = recovery; @@ -67,6 +69,11 @@ public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter { tx.activeCachesDeploymentEnabled(cacheCtx.deploymentEnabled()); } + /** {@inheritDoc} */ + @Nullable @Override public GridIntList cacheIds() { + return GridIntList.asList(cacheCtx.cacheId()); + } + /** {@inheritDoc} */ @Nullable @Override public GridCacheContext singleCacheContext(GridCacheSharedContext cctx) { return cacheCtx; @@ -288,6 +295,13 @@ public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter { return entry != null ? entry.get(0) : null; } + /** {@inheritDoc} */ + @Override public boolean mvccEnabled(GridCacheSharedContext cctx) { + GridCacheContext ctx0 = cacheCtx; + + return ctx0 != null && ctx0.mvccEnabled(); + } + /** {@inheritDoc} */ public String toString() { return S.toString(IgniteTxImplicitSingleStateImpl.class, this); 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 e6c656e0918c0..b86273f9f1a6e 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 @@ -20,9 +20,11 @@ import java.io.Externalizable; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import javax.cache.expiry.Duration; @@ -30,12 +32,12 @@ import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.failure.FailureContext; import org.apache.ignite.failure.FailureType; -import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.NodeStoppingException; -import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; @@ -56,7 +58,11 @@ import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; @@ -64,6 +70,7 @@ import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.lang.GridClosureException; import org.apache.ignite.internal.util.lang.GridTuple; @@ -152,6 +159,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig /** */ protected CacheWriteSynchronizationMode syncMode; + /** */ + private GridLongList mvccWaitTxs; + + /** Update counters map */ + private Map> updCntrs; + + /** */ + private volatile boolean qryEnlisted; + /** * Empty constructor required for {@link Externalizable}. */ @@ -212,6 +228,10 @@ protected IgniteTxLocalAdapter( txState = implicitSingle ? new IgniteTxImplicitSingleStateImpl() : new IgniteTxStateImpl(); } + public GridLongList mvccWaitTransactions() { + return mvccWaitTxs; + } + /** * @return Transaction write synchronization mode. */ @@ -283,10 +303,8 @@ protected void initResult() { return depEnabled; } - /** - * @param depEnabled Flag indicating whether deployment is enabled for caches from this transaction or not. - */ - public void activeCachesDeploymentEnabled(boolean depEnabled) { + /** {@inheritDoc} */ + @Override public void activeCachesDeploymentEnabled(boolean depEnabled) { this.depEnabled = depEnabled; } @@ -353,6 +371,8 @@ public void activeCachesDeploymentEnabled(boolean depEnabled) { * @param ret Result. */ public void implicitSingleResult(GridCacheReturn ret) { + assert ret != null; + if (ret.invokeResult()) implicitRes.mergeEntryProcessResults(ret); else @@ -476,7 +496,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A /** {@inheritDoc} */ @SuppressWarnings({"CatchGenericClass"}) - @Override public void userCommit() throws IgniteCheckedException { + @Override public final void userCommit() throws IgniteCheckedException { TransactionState state = state(); if (state != COMMITTING) { @@ -492,7 +512,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A Collection commitEntries = (near() || cctx.snapshot().needTxReadLogging()) ? allEntries() : writeEntries(); - boolean empty = F.isEmpty(commitEntries); + boolean empty = F.isEmpty(commitEntries) && !queryEnlisted(); // Register this transaction as completed prior to write-phase to // ensure proper lock ordering for removed entries. @@ -502,10 +522,14 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A cctx.tm().addCommittedTx(this); if (!empty) { + assert mvccWaitTxs == null; + batchStoreCommit(writeEntries()); WALPointer ptr = null; + Exception err = null; + cctx.database().checkpointReadLock(); try { @@ -673,10 +697,16 @@ else if (conflictCtx.isUseNew()) { CU.subjectId(this, cctx), resolveTaskName(), dhtVer, - null); + null, + mvccSnapshot()); + + if (updRes.success()) { + txEntry.updateCounter(updRes.updateCounter()); - if (updRes.success()) - txEntry.updateCounter(updRes.updatePartitionCounter()); + GridLongList waitTxs = updRes.mvccWaitTransactions(); + + updateWaitTxs(waitTxs); + } if (updRes.loggedPointer() != null) ptr = updRes.loggedPointer(); @@ -687,27 +717,28 @@ else if (conflictCtx.isUseNew()) { final GridCacheVersion dhtVer0 = dhtVer; updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerSet( - null, - eventNodeId(), - nodeId, - val0, - false, - false, - txEntry.ttl(), - false, - metrics0, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - CU.empty0(), - DR_NONE, - txEntry.conflictExpireTime(), - null, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer0, - null) + null, + eventNodeId(), + nodeId, + val0, + false, + false, + txEntry.ttl(), + false, + metrics0, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + CU.empty0(), + DR_NONE, + txEntry.conflictExpireTime(), + null, + CU.subjectId(this, cctx), + resolveTaskName(), + dhtVer0, + null, + mvccSnapshot()) ); } } @@ -724,15 +755,21 @@ else if (op == DELETE) { txEntry.oldValue(), topVer, null, - cached.detached() ? DR_NONE : drType, + cached.detached() ? DR_NONE : drType, cached.isNear() ? null : explicitVer, CU.subjectId(this, cctx), resolveTaskName(), dhtVer, - null); + null, + mvccSnapshot()); + + if (updRes.success()) { + txEntry.updateCounter(updRes.updateCounter()); + + GridLongList waitTxs = updRes.mvccWaitTransactions(); - if (updRes.success()) - txEntry.updateCounter(updRes.updatePartitionCounter()); + updateWaitTxs(waitTxs); + } if (updRes.loggedPointer() != null) ptr = updRes.loggedPointer(); @@ -742,23 +779,24 @@ else if (op == DELETE) { final GridCacheVersion dhtVer0 = dhtVer; updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerRemove( - null, - eventNodeId(), - nodeId, - false, - false, - metrics0, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - CU.empty0(), - DR_NONE, - null, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer0, - null) + null, + eventNodeId(), + nodeId, + false, + false, + metrics0, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + CU.empty0(), + DR_NONE, + null, + CU.subjectId(this, cctx), + resolveTaskName(), + dhtVer0, + null, + mvccSnapshot()) ); } } @@ -798,7 +836,7 @@ else if (op == READ) { log.debug("Ignoring READ entry when committing: " + txEntry); } else { - assert ownsLock(txEntry.cached()): + assert ownsLock(txEntry.cached()) : "Transaction does not own lock for group lock entry during commit [tx=" + this + ", txEntry=" + txEntry + ']'; @@ -839,7 +877,7 @@ assert ownsLock(txEntry.cached()): boolean isNodeStopping = X.hasCause(ex, NodeStoppingException.class); boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class); - IgniteCheckedException err = new IgniteTxHeuristicCheckedException("Failed to locally write to cache " + + IgniteCheckedException err0 = new IgniteTxHeuristicCheckedException("Failed to locally write to cache " + "(all transaction entries will be invalidated, however there was a window when " + "entries for this transaction were visible to others): " + this, ex); @@ -852,9 +890,9 @@ else if (hasInvalidEnvironmentIssue) { ", err=" + ex + ']'); } else - U.error(log, "Commit failed.", err); + U.error(log, "Commit failed.", err0); - COMMIT_ERR_UPD.compareAndSet(this, null, err); + COMMIT_ERR_UPD.compareAndSet(this, null, err0); state(UNKNOWN); @@ -876,20 +914,26 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in if (ex instanceof Error) throw ex; - throw err; + throw err0; } } + updateLocalPartitionCounters(); + if (ptr != null && !cctx.tm().logTxRecords()) cctx.wal().flush(ptr, false); } catch (StorageException e) { + err = e; + throw new IgniteCheckedException("Failed to log transaction record " + "(transaction will be rolled back): " + this, e); } finally { cctx.database().checkpointReadUnlock(); + notifyDrManager(state() == COMMITTING && err == null); + cctx.tm().resetContext(); } } @@ -909,6 +953,18 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in } } + /** + * @param waitTxs Tx ids to wait for. + */ + private void updateWaitTxs(@Nullable GridLongList waitTxs) { + if (waitTxs != null) { + if (this.mvccWaitTxs == null) + this.mvccWaitTxs = waitTxs; + else + this.mvccWaitTxs.addAll(waitTxs); + } + } + /** * Safely performs {@code updateClojure} operation on near cache entry with given {@code entryKey}. * In case of {@link GridCacheEntryRemovedException} operation will be retried. @@ -943,6 +999,7 @@ private void updateNearEntrySafely( } } + /** * Commits transaction to transaction manager. Used for one-phase commit transactions only. * @@ -1010,6 +1067,8 @@ public Collection rolledbackVersions() { @Override public void userRollback(boolean clearThreadMap) throws IgniteCheckedException { TransactionState state = state(); + notifyDrManager(false); + if (state != ROLLING_BACK && state != ROLLED_BACK) { setRollbackOnly(); @@ -1077,8 +1136,8 @@ protected IgniteCacheExpiryPolicy accessPolicy(GridCacheContext cacheCtx, Collec * @param read {@code True} if read. * @param accessTtl TTL for read operation. * @param filter Filter to check entries. - * @throws IgniteCheckedException If error. * @param computeInvoke If {@code true} computes return value for invoke operation. + * @throws IgniteCheckedException If error. */ @SuppressWarnings("unchecked") protected final void postLockWrite( @@ -1140,7 +1199,8 @@ protected final void postLockWrite( null, resolveTaskName(), null, - txEntry.keepBinary()); + txEntry.keepBinary(), + null); // TODO IGNITE-7371 } } else { @@ -1252,15 +1312,13 @@ protected final void addInvokeResult(IgniteTxEntry txEntry, res = entryProcessor.process(invokeEntry, t.get2()); - val0 = invokeEntry.value(); + val0 = invokeEntry.getValue(txEntry.keepBinary()); key0 = invokeEntry.key(); } - val0 = ctx.toCacheObject(val0); - - if (val0 != null) - ctx.validateKeyAndValue(txEntry.key(), (CacheObject)val0); + if (val0 != null) // no validation for remove case + ctx.validateKeyAndValue(txEntry.key(), ctx.toCacheObject(val0)); if (res != null) ret.addEntryProcessResult(ctx, txEntry.key(), key0, res, null, txEntry.keepBinary()); @@ -1282,14 +1340,8 @@ public boolean init() { return !txState.init(txSize) || cctx.tm().onStarted(this); } - /** - * Adds cache to the list of active caches in transaction. - * - * @param cacheCtx Cache context to add. - * @throws IgniteCheckedException If caches already enlisted in this transaction are not compatible with given - * cache (e.g. they have different stores). - */ - protected final void addActiveCache(GridCacheContext cacheCtx, boolean recovery) throws IgniteCheckedException { + /** {@inheritDoc} */ + @Override public final void addActiveCache(GridCacheContext cacheCtx, boolean recovery) throws IgniteCheckedException { txState.addActiveCache(cacheCtx, recovery, this); } @@ -1350,7 +1402,7 @@ protected void checkValid(boolean checkTimeout) throws IgniteCheckedException { * @param skipStore Skip store flag. * @return Transaction entry. */ - protected final IgniteTxEntry addEntry(GridCacheOperation op, + public final IgniteTxEntry addEntry(GridCacheOperation op, @Nullable CacheObject val, @Nullable EntryProcessor entryProcessor, Object[] invokeArgs, @@ -1503,6 +1555,13 @@ else if (explicitCand.dhtLocal()) } } + /** + * @return Map of affected partitions: cacheId -> partId. + */ + public Map> partsMap() { + return null; + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(IgniteTxLocalAdapter.class, this, "super", super.toString(), @@ -1584,6 +1643,110 @@ public long entryExpireTime(IgniteTxKey key) { return 0; } + /** + * @return Partition counters map for the given backup node. + */ + public Map updateCountersForNode(ClusterNode node) { + if (F.isEmpty(updCntrs)) + return null; + + Map res = new HashMap<>(); + + for (Map.Entry> entry : updCntrs.entrySet()) { + Map partsCntrs = entry.getValue(); + + assert !F.isEmpty(partsCntrs); + + GridCacheContext ctx0 = cctx.cacheContext(entry.getKey()); + + GridDhtPartitionsUpdateCountersMap resBackupCntrs = new GridDhtPartitionsUpdateCountersMap(); + + for (Map.Entry e : partsCntrs.entrySet()) { + Long cntr = partsCntrs.get(e.getKey()); + + if (ctx0.affinity().backupByPartition(node, e.getKey(), topologyVersionSnapshot())) { + assert cntr != null && cntr > 0 : cntr; + + resBackupCntrs.updateCounters().put(e.getKey(), cntr); + } + } + + if (!resBackupCntrs.updateCounters().isEmpty()) + res.put(entry.getKey(), resBackupCntrs); + } + + return res; + } + + /** + * @param cacheId Cache id. + * @param part Partition id. + */ + @SuppressWarnings("Java8MapApi") + public void addPartitionCountersMapping(Integer cacheId, Integer part) { + if (updCntrs == null) + updCntrs = new ConcurrentHashMap<>(); + + Map partUpdCntrs = updCntrs.get(cacheId); + + if (partUpdCntrs == null) + updCntrs.put(cacheId, partUpdCntrs = new ConcurrentHashMap<>()); + + partUpdCntrs.put(part, 0L); + } + + /** + * Merges mvcc update counters to the partition update counters. For mvcc transactions we update partitions + * counters only on commit phase. + */ + private void updateLocalPartitionCounters() { + if (F.isEmpty(updCntrs)) + return; + + for (Map.Entry> entry : updCntrs.entrySet()) { + Map partsCntrs = entry.getValue(); + + assert !F.isEmpty(partsCntrs); + + GridCacheContext ctx0 = cctx.cacheContext(entry.getKey()); + + for (Map.Entry e : partsCntrs.entrySet()) { + GridDhtLocalPartition dhtPart = ctx0.topology().localPartition(e.getKey()); + + assert dhtPart != null; + + long cntr = dhtPart.mvccUpdateCounter(); + + dhtPart.updateCounter(cntr); + + Long prev = partsCntrs.put(e.getKey(), cntr); + + assert prev == 0L : prev; + } + } + } + + /** + * @return {@code True} if there are entries, enlisted by query. + */ + public boolean queryEnlisted() { + return qryEnlisted; + } + + /** + * @param ver Mvcc version. + */ + public void markQueryEnlisted(MvccSnapshot ver) { + if (!qryEnlisted) { + if (mvccSnapshot == null) + mvccSnapshot = ver; + + cctx.coordinators().registerLocalTransaction(ver.coordinatorVersion(), ver.counter()); + + qryEnlisted = true; + } + } + /** * Post-lock closure alias. * 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 ffaaf439c8f42..d6e9d7a723a80 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 @@ -312,6 +312,16 @@ public void rollbackOnTopologyChange(AffinityTopologyVersion topVer) { } } + /** + * Rollback all active transactions with acquired Mvcc snapshot. + */ + public void rollbackMvccTxOnCoordinatorChange() { + for (IgniteInternalTx tx : activeTransactions()) { + if (tx.mvccSnapshot() != null) + ((GridNearTxLocal)tx).rollbackNearTxLocalAsync(false, false); + } + } + /** * @param cacheId Cache ID. * @param txMap Transactions map. @@ -456,6 +466,7 @@ private boolean isCompleted(IgniteInternalTx tx) { * @param concurrency Concurrency. * @param isolation Isolation. * @param timeout transaction timeout. + * @param sql Whether this transaction is being started via SQL API or not, or {@code null} if unknown. * @param txSize Expected transaction size. * @param lb Label. * @return New transaction. @@ -468,6 +479,7 @@ public GridNearTxLocal newTx( TransactionIsolation isolation, long timeout, boolean storeEnabled, + Boolean sql, int txSize, @Nullable String lb ) { @@ -487,6 +499,7 @@ public GridNearTxLocal newTx( isolation, timeout, storeEnabled, + sql, txSize, subjId, taskNameHash, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java index 87cc7ccea3635..1e0645ffb3ab5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java @@ -18,7 +18,9 @@ package org.apache.ignite.internal.processors.cache.transactions; import java.util.Collection; +import java.util.Map; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; /** @@ -50,4 +52,14 @@ public void doneRemote(GridCacheVersion baseVer, * @param cntrs Partition update indexes. */ public void setPartitionUpdateCounters(long[] cntrs); + + /** + * @param updCntrsMap Partition update counters map: cacheId -> partId -> updateCntr. + */ + public void updateCountersMap(Map updCntrsMap); + + /** + * @return Partition update counters map: cacheId -> partId -> updateCntr. + */ + public Map updateCountersMap(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java index b61a99c0301d1..4d0388b8cb035 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteSingleStateImpl.java @@ -142,4 +142,9 @@ public String toString() { return null; } + + /** {@inheritDoc} */ + @Override public boolean mvccEnabled(GridCacheSharedContext cctx) { + return entry != null && entry.context().mvccEnabled(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java index bcb900c85dd15..c1d973eeadec4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateAdapter.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; +import org.apache.ignite.internal.util.GridIntList; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.jetbrains.annotations.Nullable; @@ -31,6 +32,9 @@ * */ public abstract class IgniteTxRemoteStateAdapter implements IgniteTxRemoteState { + /** Active cache IDs. */ + private GridIntList activeCacheIds = new GridIntList(); + /** {@inheritDoc} */ @Override public boolean implicitSingle() { return false; @@ -38,9 +42,12 @@ public abstract class IgniteTxRemoteStateAdapter implements IgniteTxRemoteState /** {@inheritDoc} */ @Nullable @Override public Integer firstCacheId() { - assert false; + return activeCacheIds.isEmpty() ? null : activeCacheIds.get(0); + } - return null; + /** {@inheritDoc} */ + @Nullable @Override public GridIntList cacheIds() { + return activeCacheIds; } /** {@inheritDoc} */ @@ -67,9 +74,15 @@ public abstract class IgniteTxRemoteStateAdapter implements IgniteTxRemoteState } /** {@inheritDoc} */ - @Override public void addActiveCache(GridCacheContext cacheCtx, boolean recovery, IgniteTxLocalAdapter tx) + @Override public void addActiveCache(GridCacheContext cctx, boolean recovery, IgniteTxAdapter tx) throws IgniteCheckedException { - assert false; + assert !tx.local(); + + int cacheId = cctx.cacheId(); + + // Check if we can enlist new cache to transaction. + if (!activeCacheIds.contains(cacheId)) + activeCacheIds.add(cacheId); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java index 13264916887aa..1b6c6565fe7c2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteStateImpl.java @@ -209,4 +209,14 @@ public String toString() { return null; } + + /** {@inheritDoc} */ + @Override public boolean mvccEnabled(GridCacheSharedContext cctx) { + for (IgniteTxEntry e : writeMap.values()) { + if (e.context().mvccEnabled()) + return true; + } + + return false; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java index 1fe0d2af68028..e42fe7f88d4dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxState.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; +import org.apache.ignite.internal.util.GridIntList; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.jetbrains.annotations.Nullable; @@ -44,6 +45,13 @@ public interface IgniteTxState { */ @Nullable public Integer firstCacheId(); + /** + * Gets caches ids affected with current tx. + * + * @return tx cache ids. + */ + @Nullable public GridIntList cacheIds(); + /** * Unwind evicts for caches involved in this transaction. * @param cctx Grid cache shared context. @@ -83,7 +91,7 @@ public IgniteCheckedException validateTopology( * @param tx Transaction. * @throws IgniteCheckedException If cache check failed. */ - public void addActiveCache(GridCacheContext cacheCtx, boolean recovery, IgniteTxLocalAdapter tx) + public void addActiveCache(GridCacheContext cacheCtx, boolean recovery, IgniteTxAdapter tx) throws IgniteCheckedException; /** @@ -180,4 +188,10 @@ public void addActiveCache(GridCacheContext cacheCtx, boolean recovery, IgniteTx * @return {@code True} if transaction is empty. */ public boolean empty(); + + /** + * @param cctx Context. + * @return {@code True} if MVCC mode is enabled for transaction. + */ + public boolean mvccEnabled(GridCacheSharedContext cctx); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java index 4f14b5ce1b79c..0315f8ac88eee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java @@ -29,8 +29,6 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; import org.apache.ignite.internal.processors.cache.CacheStoppedException; -import org.apache.ignite.internal.managers.discovery.DiscoCache; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; @@ -73,6 +71,10 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter { @GridToStringInclude protected Boolean recovery; + /** */ + @GridToStringInclude + protected Boolean mvccEnabled; + /** {@inheritDoc} */ @Override public boolean implicitSingle() { return false; @@ -83,6 +85,11 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter { return activeCacheIds.isEmpty() ? null : activeCacheIds.get(0); } + /** {@inheritDoc} */ + @Nullable @Override public GridIntList cacheIds() { + return activeCacheIds; + } + /** {@inheritDoc} */ @Override public void unwindEvicts(GridCacheSharedContext cctx) { for (int i = 0; i < activeCacheIds.size(); i++) { @@ -203,8 +210,10 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter { } /** {@inheritDoc} */ - @Override public void addActiveCache(GridCacheContext cacheCtx, boolean recovery, IgniteTxLocalAdapter tx) + @Override public void addActiveCache(GridCacheContext cacheCtx, boolean recovery, IgniteTxAdapter tx) throws IgniteCheckedException { + assert tx.local(); + GridCacheSharedContext cctx = cacheCtx.shared(); int cacheId = cacheCtx.cacheId(); @@ -215,6 +224,12 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter { this.recovery = recovery; + if (this.mvccEnabled != null && this.mvccEnabled != cacheCtx.mvccEnabled()) + throw new IgniteCheckedException("Failed to enlist new cache to existing transaction " + + "(caches with different mvcc settings can't be enlisted in one transaction)."); + + this.mvccEnabled = cacheCtx.mvccEnabled(); + // Check if we can enlist new cache to transaction. if (!activeCacheIds.contains(cacheId)) { String err = cctx.verifyTxCompatibility(tx, activeCacheIds, cacheCtx); @@ -461,6 +476,11 @@ public synchronized Collection allEntriesCopy() { return writeView != null && writeView.size() == 1 ? F.firstValue(writeView) : null; } + /** {@inheritDoc} */ + @Override public boolean mvccEnabled(GridCacheSharedContext cctx) { + return Boolean.TRUE == mvccEnabled; + } + /** {@inheritDoc} */ public String toString() { return S.toString(IgniteTxStateImpl.class, this, "txMap", allEntriesCopy()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java index d5b28ef97c5f4..65944517c445c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java @@ -17,7 +17,9 @@ package org.apache.ignite.internal.processors.cache.tree; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; @@ -26,6 +28,8 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.lang.IgniteInClosure; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccVersionIsValid; + /** * */ @@ -36,61 +40,126 @@ public abstract class AbstractDataInnerIO extends BPlusInnerIO i * @param canGetRow If we can get full row from this page. * @param itemSize Single item size on page. */ - AbstractDataInnerIO(int type, int ver, boolean canGetRow, int itemSize) { + protected AbstractDataInnerIO(int type, int ver, boolean canGetRow, int itemSize) { super(type, ver, canGetRow, itemSize); } /** {@inheritDoc} */ - @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) { + @Override public final void storeByOffset(long pageAddr, int off, CacheSearchRow row) { assert row.link() != 0; PageUtils.putLong(pageAddr, off, row.link()); - PageUtils.putInt(pageAddr, off + 8, row.hash()); + off += 8; + + PageUtils.putInt(pageAddr, off, row.hash()); + off += 4; if (storeCacheId()) { assert row.cacheId() != CU.UNDEFINED_CACHE_ID : row; - PageUtils.putInt(pageAddr, off + 12, row.cacheId()); + PageUtils.putInt(pageAddr, off, row.cacheId()); + off += 4; + } + + if (storeMvccVersion()) { + long mvccCrd = row.mvccCoordinatorVersion(); + long mvccCntr = row.mvccCounter(); + int opCntr = row.mvccOperationCounter(); + + assert mvccVersionIsValid(mvccCrd, mvccCntr, opCntr); + + PageUtils.putLong(pageAddr, off, mvccCrd); + off += 8; + + PageUtils.putLong(pageAddr, off, mvccCntr); + off += 8; + + PageUtils.putInt(pageAddr, off, opCntr); } } /** {@inheritDoc} */ - @Override public CacheSearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) { - int cacheId = getCacheId(pageAddr, idx); - int hash = getHash(pageAddr, idx); + @Override public final CacheSearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) + throws IgniteCheckedException { long link = getLink(pageAddr, idx); + int hash = getHash(pageAddr, idx); + + int cacheId = storeCacheId() ? getCacheId(pageAddr, idx) : CU.UNDEFINED_CACHE_ID; + + if (storeMvccVersion()) { + long mvccCrd = getMvccCoordinatorVersion(pageAddr, idx); + long mvccCntr = getMvccCounter(pageAddr, idx); + int opCntr = getMvccOperationCounter(pageAddr, idx); + + assert mvccVersionIsValid(mvccCrd, mvccCntr, opCntr); + + return ((CacheDataTree)tree).rowStore().mvccRow(cacheId, + hash, + link, + CacheDataRowAdapter.RowData.KEY_ONLY, + mvccCrd, + mvccCntr, + opCntr); + } return ((CacheDataTree)tree).rowStore().keySearchRow(cacheId, hash, link); } /** {@inheritDoc} */ - @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, - int srcIdx) { - int hash = ((RowLinkIO)srcIo).getHash(srcPageAddr, srcIdx); - long link = ((RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx); + @Override public final void store(long dstPageAddr, + int dstIdx, + BPlusIO srcIo, + long srcPageAddr, + int srcIdx) + { + RowLinkIO rowIo = ((RowLinkIO)srcIo); + + long link =rowIo.getLink(srcPageAddr, srcIdx); + int hash = rowIo.getHash(srcPageAddr, srcIdx); + int off = offset(dstIdx); PageUtils.putLong(dstPageAddr, off, link); - PageUtils.putInt(dstPageAddr, off + 8, hash); + off += 8; + + PageUtils.putInt(dstPageAddr, off, hash); + off += 4; if (storeCacheId()) { - int cacheId = ((RowLinkIO)srcIo).getCacheId(srcPageAddr, srcIdx); + int cacheId = rowIo.getCacheId(srcPageAddr, srcIdx); assert cacheId != CU.UNDEFINED_CACHE_ID; - PageUtils.putInt(dstPageAddr, off + 12, cacheId); + PageUtils.putInt(dstPageAddr, off, cacheId); + off += 4; + } + + if (storeMvccVersion()) { + long mvccCrd = rowIo.getMvccCoordinatorVersion(srcPageAddr, srcIdx); + long mvccCntr = rowIo.getMvccCounter(srcPageAddr, srcIdx); + int opCntr = rowIo.getMvccOperationCounter(srcPageAddr, srcIdx); + + assert MvccUtils.mvccVersionIsValid(mvccCrd, mvccCntr, opCntr); + + PageUtils.putLong(dstPageAddr, off, mvccCrd); + off += 8; + + PageUtils.putLong(dstPageAddr, off, mvccCntr); + off += 8; + + PageUtils.putInt(dstPageAddr, off, opCntr); } } /** {@inheritDoc} */ - @Override public long getLink(long pageAddr, int idx) { + @Override public final long getLink(long pageAddr, int idx) { assert idx < getCount(pageAddr) : idx; return PageUtils.getLong(pageAddr, offset(idx)); } /** {@inheritDoc} */ - @Override public int getHash(long pageAddr, int idx) { + @Override public final int getHash(long pageAddr, int idx) { return PageUtils.getInt(pageAddr, offset(idx) + 8); } @@ -105,5 +174,14 @@ public abstract class AbstractDataInnerIO extends BPlusInnerIO i /** * @return {@code True} if cache ID has to be stored. */ - protected abstract boolean storeCacheId(); + protected boolean storeCacheId() { + return false; + } + + /** + * @return {@code True} if mvcc version has to be stored. + */ + protected boolean storeMvccVersion() { + return false; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java index e9a3a9b503dae..f0e5c90e375cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.tree; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; @@ -26,6 +27,8 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.lang.IgniteInClosure; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccVersionIsValid; + /** * */ @@ -35,7 +38,7 @@ public abstract class AbstractDataLeafIO extends BPlusLeafIO imp * @param ver Page format version. * @param itemSize Single item size on page. */ - AbstractDataLeafIO(int type, int ver, int itemSize) { + public AbstractDataLeafIO(int type, int ver, int itemSize) { super(type, ver, itemSize); } @@ -44,52 +47,133 @@ public abstract class AbstractDataLeafIO extends BPlusLeafIO imp assert row.link() != 0; PageUtils.putLong(pageAddr, off, row.link()); - PageUtils.putInt(pageAddr, off + 8, row.hash()); + off += 8; + + PageUtils.putInt(pageAddr, off, row.hash()); + off += 4; if (storeCacheId()) { assert row.cacheId() != CU.UNDEFINED_CACHE_ID; - PageUtils.putInt(pageAddr, off + 12, row.cacheId()); + PageUtils.putInt(pageAddr, off, row.cacheId()); + off += 4; + } + + if (storeMvccVersion()) { + long mvccCrdVer = row.mvccCoordinatorVersion(); + long mvccCntr = row.mvccCounter(); + int mvccOpCntr = row.mvccOperationCounter(); + + assert mvccVersionIsValid(mvccCrdVer, mvccCntr, mvccOpCntr); + + PageUtils.putLong(pageAddr, off, mvccCrdVer); + off += 8; + + PageUtils.putLong(pageAddr, off, mvccCntr); + off += 8; + + PageUtils.putInt(pageAddr, off, mvccOpCntr); + off += 4; + + // Lock version the same as mvcc version, a new row is + // always locked by Tx, in scope of which it was created. + PageUtils.putLong(pageAddr, off, mvccCrdVer); + off += 8; + + PageUtils.putLong(pageAddr, off, mvccCntr); } } /** {@inheritDoc} */ @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, - int srcIdx) { - int hash = ((RowLinkIO)srcIo).getHash(srcPageAddr, srcIdx); - long link = ((RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx); + int srcIdx) { + RowLinkIO rowIo = (RowLinkIO) srcIo; + + long link = rowIo.getLink(srcPageAddr, srcIdx); + int hash = rowIo.getHash(srcPageAddr, srcIdx); + int off = offset(dstIdx); PageUtils.putLong(dstPageAddr, off, link); - PageUtils.putInt(dstPageAddr, off + 8, hash); + off += 8; + + PageUtils.putInt(dstPageAddr, off, hash); + off += 4; if (storeCacheId()) { - int cacheId = ((RowLinkIO)srcIo).getCacheId(srcPageAddr, srcIdx); + int cacheId = rowIo.getCacheId(srcPageAddr, srcIdx); assert cacheId != CU.UNDEFINED_CACHE_ID; - PageUtils.putInt(dstPageAddr, off + 12, cacheId); + PageUtils.putInt(dstPageAddr, off, cacheId); + off += 4; + } + + if (storeMvccVersion()) { + long mvccCrd = rowIo.getMvccCoordinatorVersion(srcPageAddr, srcIdx); + long mvccCntr = rowIo.getMvccCounter(srcPageAddr, srcIdx); + int mvccOpCntr = rowIo.getMvccOperationCounter(srcPageAddr, srcIdx); + + assert mvccVersionIsValid(mvccCrd, mvccCntr, mvccOpCntr); + + long lockCrdVer = rowIo.getMvccLockCoordinatorVersion(srcPageAddr, srcIdx); + long lockCntr = rowIo.getMvccLockCounter(srcPageAddr, srcIdx); + + // Lock version cannot be blank + assert mvccVersionIsValid(lockCrdVer, lockCntr); + + PageUtils.putLong(dstPageAddr, off, mvccCrd); + off += 8; + + PageUtils.putLong(dstPageAddr, off, mvccCntr); + off += 8; + + PageUtils.putInt(dstPageAddr, off, mvccOpCntr); + off += 4; + + PageUtils.putLong(dstPageAddr, off, lockCrdVer); + off += 8; + + PageUtils.putLong(dstPageAddr, off, lockCntr); } } /** {@inheritDoc} */ - @Override public CacheSearchRow getLookupRow(BPlusTree tree, long buf, int idx) { - int cacheId = getCacheId(buf, idx); - int hash = getHash(buf, idx); - long link = getLink(buf, idx); + @Override public final CacheSearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) + throws IgniteCheckedException { + long link = getLink(pageAddr, idx); + int hash = getHash(pageAddr, idx); + + int cacheId = storeCacheId() ? getCacheId(pageAddr, idx) : CU.UNDEFINED_CACHE_ID; + + if (storeMvccVersion()) { + long mvccCrd = getMvccCoordinatorVersion(pageAddr, idx); + long mvccCntr = getMvccCounter(pageAddr, idx); + int mvccOpCntr = getMvccOperationCounter(pageAddr, idx); + + assert mvccVersionIsValid(mvccCrd, mvccCntr, mvccOpCntr); + + return ((CacheDataTree)tree).rowStore().mvccRow(cacheId, + hash, + link, + CacheDataRowAdapter.RowData.KEY_ONLY, + mvccCrd, + mvccCntr, + mvccOpCntr); + } return ((CacheDataTree)tree).rowStore().keySearchRow(cacheId, hash, link); } /** {@inheritDoc} */ - @Override public long getLink(long pageAddr, int idx) { + @Override public final long getLink(long pageAddr, int idx) { assert idx < getCount(pageAddr) : idx; return PageUtils.getLong(pageAddr, offset(idx)); } /** {@inheritDoc} */ - @Override public int getHash(long pageAddr, int idx) { + @Override public final int getHash(long pageAddr, int idx) { return PageUtils.getInt(pageAddr, offset(idx) + 8); } @@ -104,5 +188,14 @@ public abstract class AbstractDataLeafIO extends BPlusLeafIO imp /** * @return {@code True} if cache ID has to be stored. */ - protected abstract boolean storeCacheId(); + protected boolean storeCacheId() { + return false; + } + + /** + * @return {@code True} if mvcc version has to be stored. + */ + protected boolean storeMvccVersion() { + return false; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataRowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataRowStore.java index 28d8919aa338e..7514798b110a4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataRowStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataRowStore.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; import org.apache.ignite.internal.processors.cache.persistence.RowStore; import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; import org.apache.ignite.internal.util.typedef.internal.CU; /** @@ -54,12 +55,30 @@ public CacheDataRowStore(CacheGroupContext grp, FreeList freeList, int partId) { * @return Search row. */ CacheSearchRow keySearchRow(int cacheId, int hash, long link) { - DataRow dataRow = new DataRow(grp, hash, link, partId, CacheDataRowAdapter.RowData.KEY_ONLY); + return initDataRow(new DataRow(grp, hash, link, partId, CacheDataRowAdapter.RowData.KEY_ONLY), cacheId); + } - if (dataRow.cacheId() == CU.UNDEFINED_CACHE_ID && grp.sharedGroup()) - dataRow.cacheId(cacheId); + /** + * @param cacheId Cache ID. + * @param hash Hash code. + * @param link Link. + * @param rowData Required row data. + * @param crdVer Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @param opCntr Mvcc operation counter. + * @return Search row. + */ + MvccDataRow mvccRow(int cacheId, int hash, long link, CacheDataRowAdapter.RowData rowData, long crdVer, long mvccCntr, int opCntr) { + MvccDataRow dataRow = new MvccDataRow(grp, + hash, + link, + partId, + rowData, + crdVer, + mvccCntr, + opCntr); - return dataRow; + return initDataRow(dataRow, cacheId); } /** @@ -70,8 +89,14 @@ CacheSearchRow keySearchRow(int cacheId, int hash, long link) { * @return Data row. */ CacheDataRow dataRow(int cacheId, int hash, long link, CacheDataRowAdapter.RowData rowData) { - DataRow dataRow = new DataRow(grp, hash, link, partId, rowData); + return initDataRow(new DataRow(grp, hash, link, partId, rowData), cacheId); + } + /** + * @param dataRow Data row. + * @param cacheId Cache ID. + */ + private T initDataRow(T dataRow, int cacheId) { if (dataRow.cacheId() == CU.UNDEFINED_CACHE_ID && grp.sharedGroup()) dataRow.cacheId(cacheId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java index c6214ebe6aa6a..4226ec463c066 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; @@ -28,12 +29,19 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPagePayload; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccCacheIdAwareDataInnerIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccCacheIdAwareDataLeafIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataInnerIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataLeafIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.internal.CU; import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId; import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE; /** * @@ -46,7 +54,7 @@ public class CacheDataTree extends BPlusTree { private final CacheGroupContext grp; /** - * @param grp Ccahe group. + * @param grp Cache group. * @param name Tree name. * @param reuseList Reuse list. * @param rowStore Row store. @@ -69,8 +77,8 @@ public CacheDataTree( grp.offheap().globalRemoveId(), metaPageId, reuseList, - grp.sharedGroup() ? CacheIdAwareDataInnerIO.VERSIONS : DataInnerIO.VERSIONS, - grp.sharedGroup() ? CacheIdAwareDataLeafIO.VERSIONS : DataLeafIO.VERSIONS, + innerIO(grp), + leafIO(grp), grp.shared().kernalContext().failure()); assert rowStore != null; @@ -83,16 +91,41 @@ public CacheDataTree( initTree(initNew); } + /** + * @param grp Cache group. + * @return Tree inner IO. + */ + private static IOVersions innerIO(CacheGroupContext grp) { + if (grp.mvccEnabled()) + return grp.sharedGroup() ? MvccCacheIdAwareDataInnerIO.VERSIONS : MvccDataInnerIO.VERSIONS; + + return grp.sharedGroup() ? CacheIdAwareDataInnerIO.VERSIONS : DataInnerIO.VERSIONS; + } + + /** + * @param grp Cache group. + * @return Tree leaf IO. + */ + private static IOVersions leafIO(CacheGroupContext grp) { + if (grp.mvccEnabled()) + return grp.sharedGroup() ? MvccCacheIdAwareDataLeafIO.VERSIONS : MvccDataLeafIO.VERSIONS; + + return grp.sharedGroup() ? CacheIdAwareDataLeafIO.VERSIONS : DataLeafIO.VERSIONS; + } + /** * @return Row store. */ - CacheDataRowStore rowStore() { + public CacheDataRowStore rowStore() { return rowStore; } /** {@inheritDoc} */ @Override protected int compare(BPlusIO iox, long pageAddr, int idx, CacheSearchRow row) throws IgniteCheckedException { + assert !grp.mvccEnabled() || row.mvccCoordinatorVersion() != 0 + || (row.getClass() == SearchRow.class && row.key() == null) : row; + RowLinkIO io = (RowLinkIO)iox; int cmp; @@ -102,8 +135,6 @@ CacheDataRowStore rowStore() { int cacheId = io.getCacheId(pageAddr, idx); - assert cacheId != CU.UNDEFINED_CACHE_ID : "Cache ID is not stored"; - cmp = Integer.compare(cacheId, row.cacheId()); if (cmp != 0) @@ -130,21 +161,43 @@ CacheDataRowStore rowStore() { assert row.key() != null : row; - return compareKeys(row.key(), link); + cmp = compareKeys(row.key(), link); + + if (cmp != 0 || !grp.mvccEnabled()) + return cmp; + + long crd = io.getMvccCoordinatorVersion(pageAddr, idx); + long cntr = io.getMvccCounter(pageAddr, idx); + int opCntr = io.getMvccOperationCounter(pageAddr, idx); + + assert MvccUtils.mvccVersionIsValid(crd, cntr, opCntr); + + return -MvccUtils.compare(crd, cntr, opCntr, row); // descending order } /** {@inheritDoc} */ - @Override protected CacheDataRow getRow(BPlusIO io, long pageAddr, int idx, Object flags) + @Override public CacheDataRow getRow(BPlusIO io, long pageAddr, int idx, Object flags) throws IgniteCheckedException { - long link = ((RowLinkIO)io).getLink(pageAddr, idx); - int hash = ((RowLinkIO)io).getHash(pageAddr, idx); - int cacheId = ((RowLinkIO)io).getCacheId(pageAddr, idx); + RowLinkIO rowIo = (RowLinkIO)io; + + long link = rowIo.getLink(pageAddr, idx); + int hash = rowIo.getHash(pageAddr, idx); + + int cacheId = grp.sharedGroup() ? rowIo.getCacheId(pageAddr, idx) : CU.UNDEFINED_CACHE_ID; CacheDataRowAdapter.RowData x = flags != null ? (CacheDataRowAdapter.RowData)flags : CacheDataRowAdapter.RowData.FULL; - return rowStore.dataRow(cacheId, hash, link, x); + if (grp.mvccEnabled()) { + long mvccCrdVer = rowIo.getMvccCoordinatorVersion(pageAddr, idx); + long mvccCntr = rowIo.getMvccCounter(pageAddr, idx); + int mvccOpCntr = rowIo.getMvccOperationCounter(pageAddr, idx); + + return rowStore.mvccRow(cacheId, hash, link, x, mvccCrdVer, mvccCntr, mvccOpCntr); + } + else + return rowStore.dataRow(cacheId, hash, link, x); } /** @@ -174,6 +227,9 @@ private int compareKeys(KeyCacheObject key, final long link) throws IgniteChecke if (data.nextLink() == 0) { long addr = pageAddr + data.offset(); + if (grp.mvccEnabled()) + addr += MVCC_INFO_SIZE; // Skip MVCC info. + if (grp.storeCacheIdInDataPage()) addr += 4; // Skip cache id. @@ -220,7 +276,7 @@ private int compareKeys(KeyCacheObject key, final long link) throws IgniteChecke } // TODO GG-11768. - CacheDataRowAdapter other = new CacheDataRowAdapter(link); + CacheDataRowAdapter other = grp.mvccEnabled() ? new MvccDataRow(link) : new CacheDataRowAdapter(link); other.initFromLink(grp, CacheDataRowAdapter.RowData.KEY_ONLY); byte[] bytes1 = other.key().valueBytes(grp.cacheObjectContext()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java index acb42a002e65b..65d90e9c0a5e7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java @@ -37,12 +37,12 @@ private CacheIdAwareDataInnerIO(int ver) { } /** {@inheritDoc} */ - @Override public int getCacheId(long pageAddr, int idx) { - return PageUtils.getInt(pageAddr, offset(idx) + 12); + @Override protected boolean storeCacheId() { + return true; } /** {@inheritDoc} */ - @Override protected boolean storeCacheId() { - return true; + @Override public int getCacheId(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 12); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java index 72631682ae8ca..70f0f84375d0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java @@ -37,12 +37,12 @@ private CacheIdAwareDataLeafIO(int ver) { } /** {@inheritDoc} */ - @Override public int getCacheId(long pageAddr, int idx) { - return PageUtils.getInt(pageAddr, offset(idx) + 12); + @Override protected boolean storeCacheId() { + return true; } /** {@inheritDoc} */ - @Override protected boolean storeCacheId() { - return true; + @Override public int getCacheId(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 12); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java index 8625338e4a2c8..cd8debb21a3f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.tree; import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; -import org.apache.ignite.internal.util.typedef.internal.CU; /** * @@ -35,14 +34,4 @@ public final class DataInnerIO extends AbstractDataInnerIO { private DataInnerIO(int ver) { super(T_DATA_REF_INNER, ver, true, 12); } - - /** {@inheritDoc} */ - @Override public int getCacheId(long pageAddr, int idx) { - return CU.UNDEFINED_CACHE_ID; - } - - /** {@inheritDoc} */ - @Override protected boolean storeCacheId() { - return false; - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java index d53964fdb7753..3ba9619d13083 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.tree; import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; -import org.apache.ignite.internal.util.typedef.internal.CU; /** * @@ -35,14 +34,4 @@ public final class DataLeafIO extends AbstractDataLeafIO { private DataLeafIO(int ver) { super(T_DATA_REF_LEAF, ver, 12); } - - /** {@inheritDoc} */ - @Override public int getCacheId(long pageAddr, int idx) { - return CU.UNDEFINED_CACHE_ID; - } - - /** {@inheritDoc} */ - @Override protected boolean storeCacheId() { - return false; - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java index 29bbaafdf78e9..806f030c53549 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java @@ -42,7 +42,7 @@ public class DataRow extends CacheDataRowAdapter { * @param part Partition. * @param rowData Required row data. */ - DataRow(CacheGroupContext grp, int hash, long link, int part, RowData rowData) { + protected DataRow(CacheGroupContext grp, int hash, long link, int part, RowData rowData) { super(link); this.hash = hash; @@ -50,8 +50,9 @@ public class DataRow extends CacheDataRowAdapter { this.part = part; try { - // We can not init data row lazily because underlying buffer can be concurrently cleared. - initFromLink(grp, rowData); + // We can not init data row lazily outside of entry lock because underlying buffer can be concurrently cleared. + if (rowData != RowData.LINK_ONLY) + initFromLink(grp, rowData); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -81,6 +82,27 @@ public DataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, int pa this.cacheId = cacheId; } + /** + * @param link Link. + */ + protected DataRow(long link) { + super(link); + } + + /** + * + */ + DataRow() { + super(0); + } + + /** {@inheritDoc} */ + @Override public void key(KeyCacheObject key) { + super.key(key); + + hash = key.hashCode(); + } + /** {@inheritDoc} */ @Override public int partition() { return part; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java index 78a9f56e8b61d..554b86ad1dd83 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java @@ -111,7 +111,7 @@ public PendingEntriesTree( } /** {@inheritDoc} */ - @Override protected PendingRow getRow(BPlusIO io, long pageAddr, int idx, Object flag) + @Override public PendingRow getRow(BPlusIO io, long pageAddr, int idx, Object flag) throws IgniteCheckedException { PendingRow row = io.getLookupRow(this, pageAddr, idx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingRow.java index 1a6a1837c1bff..4116ae0b21957 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingRow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingRow.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -68,7 +69,7 @@ public PendingRow(int cacheId, long expireTime, long link) { * @throws IgniteCheckedException If failed. */ PendingRow initKey(CacheGroupContext grp) throws IgniteCheckedException { - CacheDataRowAdapter rowData = new CacheDataRowAdapter(link); + CacheDataRowAdapter rowData = grp.mvccEnabled() ? new MvccDataRow(link) : new CacheDataRowAdapter(link); rowData.initFromLink(grp, CacheDataRowAdapter.RowData.KEY_ONLY); key = rowData.key(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java index 55f880cfceaf9..84873510d0af3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java @@ -40,5 +40,70 @@ public interface RowLinkIO { * @param idx Index. * @return Cache ID or {@code 0} if cache ID is not defined. */ - public int getCacheId(long pageAddr, int idx); + default int getCacheId(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc coordinator version. + */ + default long getMvccCoordinatorVersion(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc operation counter. + */ + default int getMvccOperationCounter(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc counter. + */ + default long getMvccCounter(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc coordinator version. + */ + default long getMvccLockCoordinatorVersion(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc counter. + */ + default long getMvccLockCounter(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @param lockCrd Mvcc lock coordinator version. + */ + default void setMvccLockCoordinatorVersion(long pageAddr, int idx, long lockCrd) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @param lockCntr Mvcc lock counter. + */ + default void setMvccLockCounter(long pageAddr, int idx, long lockCntr) { + throw new UnsupportedOperationException(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/SearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/SearchRow.java index 62570139dedd8..4b166d4fcd4ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/SearchRow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/SearchRow.java @@ -18,7 +18,13 @@ package org.apache.ignite.internal.processors.cache.tree; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.util.typedef.internal.S; + +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; /** * @@ -73,4 +79,29 @@ public SearchRow(int cacheId) { @Override public int cacheId() { return cacheId; } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return MVCC_CRD_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return MVCC_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return MVCC_OP_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public byte mvccTxState() { + return TxState.NA; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SearchRow.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataInnerIO.java new file mode 100644 index 0000000000000..32cd8bad4d51c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataInnerIO.java @@ -0,0 +1,68 @@ +/* + * 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.tree.mvcc.data; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; +import org.apache.ignite.internal.processors.cache.tree.AbstractDataInnerIO; + +/** + * + */ +public final class MvccCacheIdAwareDataInnerIO extends AbstractDataInnerIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>( + new MvccCacheIdAwareDataInnerIO(1) + ); + + /** + * @param ver Page format version. + */ + private MvccCacheIdAwareDataInnerIO(int ver) { + super(T_CACHE_ID_DATA_REF_MVCC_INNER, ver, true, 36); + } + + /** {@inheritDoc} */ + @Override protected boolean storeCacheId() { + return true; + } + + /** {@inheritDoc} */ + @Override protected boolean storeMvccVersion() { + return true; + } + + /** {@inheritDoc} */ + @Override public int getCacheId(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 12); + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 16); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 24); + } + + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 32); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataLeafIO.java new file mode 100644 index 0000000000000..3da772933a8bd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccCacheIdAwareDataLeafIO.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.tree.mvcc.data; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; +import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO; + +/** + * + */ +public final class MvccCacheIdAwareDataLeafIO extends AbstractDataLeafIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>( + new MvccCacheIdAwareDataLeafIO(1) + ); + + /** + * @param ver Page format version. + */ + private MvccCacheIdAwareDataLeafIO(int ver) { + super(T_CACHE_ID_DATA_REF_MVCC_LEAF, ver, 52); + } + + /** {@inheritDoc} */ + @Override protected boolean storeCacheId() { + return true; + } + + /** {@inheritDoc} */ + @Override protected boolean storeMvccVersion() { + return true; + } + + /** {@inheritDoc} */ + @Override public int getCacheId(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 12); + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 16); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 24); + } + + /** {@inheritDoc} */ + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 32); + } + + /** {@inheritDoc} */ + @Override public long getMvccLockCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 36); + } + + /** {@inheritDoc} */ + @Override public long getMvccLockCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 44); + } + + /** {@inheritDoc} */ + @Override public void setMvccLockCoordinatorVersion(long pageAddr, int idx, long lockCrd) { + PageUtils.putLong(pageAddr, offset(idx) + 36, lockCrd); + } + + /** {@inheritDoc} */ + @Override public void setMvccLockCounter(long pageAddr, int idx, long lockCntr) { + PageUtils.putLong(pageAddr, offset(idx) + 44, lockCntr); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataInnerIO.java new file mode 100644 index 0000000000000..0b2f91c9763d8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataInnerIO.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.cache.tree.mvcc.data; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; +import org.apache.ignite.internal.processors.cache.tree.AbstractDataInnerIO; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.lang.IgniteInClosure; + +/** + * + */ +public final class MvccDataInnerIO extends AbstractDataInnerIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>( + new MvccDataInnerIO(1) + ); + + /** + * @param ver Page format version. + */ + private MvccDataInnerIO(int ver) { + super(T_DATA_REF_MVCC_INNER, ver, true, 32); + } + + /** {@inheritDoc} */ + @Override public void visit(long pageAddr, IgniteInClosure c) { + int cnt = getCount(pageAddr); + + for (int i = 0; i < cnt; i++) + c.apply(new MvccDataRow(getLink(pageAddr, i))); + } + + /** {@inheritDoc} */ + @Override protected boolean storeMvccVersion() { + return true; + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 12); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 20); + } + + /** {@inheritDoc} */ + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 28); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataLeafIO.java new file mode 100644 index 0000000000000..ab498d9d4c3be --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataLeafIO.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.tree.mvcc.data; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; +import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO; +import org.apache.ignite.lang.IgniteInClosure; + +/** + * + */ +public final class MvccDataLeafIO extends AbstractDataLeafIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>( + new MvccDataLeafIO(1) + ); + + /** + * @param ver Page format version. + */ + private MvccDataLeafIO(int ver) { + super(T_DATA_REF_MVCC_LEAF, ver, 48); + } + + /** {@inheritDoc} */ + @Override public void visit(long pageAddr, IgniteInClosure c) { + int cnt = getCount(pageAddr); + + for (int i = 0; i < cnt; i++) + c.apply(new MvccDataRow(getLink(pageAddr, i))); + } + + /** {@inheritDoc} */ + @Override protected boolean storeMvccVersion() { + return true; + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 12); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 20); + } + + /** {@inheritDoc} */ + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 28); + } + + /** {@inheritDoc} */ + @Override public long getMvccLockCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 32); + } + + /** {@inheritDoc} */ + @Override public long getMvccLockCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 40); + } + + /** {@inheritDoc} */ + @Override public void setMvccLockCoordinatorVersion(long pageAddr, int idx, long lockCrd) { + PageUtils.putLong(pageAddr, offset(idx) + 32, lockCrd); + } + + /** {@inheritDoc} */ + @Override public void setMvccLockCounter(long pageAddr, int idx, long lockCntr) { + PageUtils.putLong(pageAddr, offset(idx) + 40, lockCntr); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java new file mode 100644 index 0000000000000..7dc205127d44e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java @@ -0,0 +1,263 @@ +/* + * 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.tree.mvcc.data; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionImpl; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; +import org.apache.ignite.internal.processors.cache.tree.DataRow; +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 static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_BIT_OFF; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_MASK; + +/** + * + */ +public class MvccDataRow extends DataRow { + /** Mvcc coordinator version. */ + @GridToStringInclude + protected long mvccCrd; + + /** Mvcc counter. */ + @GridToStringInclude + protected long mvccCntr; + + /** Mvcc operation counter. */ + @GridToStringInclude + protected int mvccOpCntr; + + /** Mvcc tx state. */ + @GridToStringInclude + protected byte mvccTxState; + + /** New mvcc coordinator version. */ + @GridToStringInclude + protected long newMvccCrd; + + /** New mvcc counter. */ + @GridToStringInclude + protected long newMvccCntr; + + /** New mvcc operation counter. */ + @GridToStringInclude + protected int newMvccOpCntr; + + /** New mvcc tx state. */ + @GridToStringInclude + protected byte newMvccTxState; + + /** + * @param link Link. + */ + public MvccDataRow(long link) { + super(link); + } + + /** + * @param grp Context. + * @param hash Key hash. + * @param link Link. + * @param part Partition number. + * @param rowData Data. + * @param crdVer Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @param mvccOpCntr Mvcc operation counter. + */ + public MvccDataRow(CacheGroupContext grp, + int hash, + long link, + int part, + RowData rowData, + long crdVer, + long mvccCntr, + int mvccOpCntr) { + super(grp, hash, link, part, rowData); + + assert MvccUtils.mvccVersionIsValid(crdVer, mvccCntr, mvccOpCntr); + + assert rowData == RowData.LINK_ONLY + || this.mvccCrd == crdVer && this.mvccCntr == mvccCntr && this.mvccOpCntr == mvccOpCntr : + "mvccVer=" + new MvccVersionImpl(crdVer, mvccCntr, mvccOpCntr) + + ", dataMvccVer=" + new MvccVersionImpl(this.mvccCrd, this.mvccCntr, this.mvccOpCntr) ; + + if (rowData == RowData.LINK_ONLY) { + this.mvccCrd = crdVer; + this.mvccCntr = mvccCntr; + this.mvccOpCntr = mvccOpCntr; + } + } + + /** + * @param key Key. + * @param val Value. + * @param ver Version. + * @param part Partition. + * @param expireTime Expire time. + * @param cacheId Cache ID. + * @param mvccVer Mvcc version. + * @param newMvccVer New mvcc version. + */ + public MvccDataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, int part, long expireTime, int cacheId, + MvccVersion mvccVer, MvccVersion newMvccVer) { + super(key, val, ver, part, expireTime, cacheId); + + this.mvccCrd = mvccVer.coordinatorVersion(); + this.mvccCntr = mvccVer.counter(); + this.mvccOpCntr = mvccVer.operationCounter(); + + if (newMvccVer == null) { + newMvccCrd = MVCC_CRD_COUNTER_NA; + newMvccCntr = MVCC_COUNTER_NA; + newMvccOpCntr = MVCC_OP_COUNTER_NA; + } + else { + newMvccCrd = newMvccVer.coordinatorVersion(); + newMvccCntr = newMvccVer.counter(); + newMvccOpCntr = newMvccVer.operationCounter(); + } + } + + /** {@inheritDoc} */ + @Override protected int readHeader(long addr, int off) { + // xid_min. + mvccCrd = PageUtils.getLong(addr, off); + mvccCntr = PageUtils.getLong(addr, off + 8); + + int withHint = PageUtils.getInt(addr, off + 16); + + mvccOpCntr = withHint & ~MVCC_HINTS_MASK; + mvccTxState = (byte)(withHint >>> MVCC_HINTS_BIT_OFF); + + assert MvccUtils.mvccVersionIsValid(mvccCrd, mvccCntr, mvccOpCntr); + + // xid_max. + newMvccCrd = PageUtils.getLong(addr, off + 20); + newMvccCntr = PageUtils.getLong(addr, off + 28); + + withHint = PageUtils.getInt(addr, off + 36); + + newMvccOpCntr = withHint & ~MVCC_HINTS_MASK; + newMvccTxState = (byte)(withHint >>> MVCC_HINTS_BIT_OFF); + + assert newMvccCrd == MVCC_CRD_COUNTER_NA || MvccUtils.mvccVersionIsValid(newMvccCrd, newMvccCntr, newMvccOpCntr); + + return MVCC_INFO_SIZE; + } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return mvccCrd; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return mvccCntr; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return mvccOpCntr; + } + + /** {@inheritDoc} */ + public byte mvccTxState() { + return mvccTxState; + } + + /** {@inheritDoc} */ + @Override public long newMvccCoordinatorVersion() { + return newMvccCrd; + } + + /** {@inheritDoc} */ + @Override public long newMvccCounter() { + return newMvccCntr; + } + + /** {@inheritDoc} */ + @Override public int newMvccOperationCounter() { + return newMvccOpCntr; + } + + /** {@inheritDoc} */ + @Override public byte newMvccTxState() { + return newMvccTxState; + } + + /** {@inheritDoc} */ + @Override public void newMvccVersion(long crd, long cntr, int opCntr) { + newMvccCrd = crd; + newMvccCntr = cntr; + newMvccOpCntr = opCntr; + + // reset tx state + newMvccTxState = TxState.NA; + } + + /** {@inheritDoc} */ + @Override public void mvccVersion(long crd, long cntr, int opCntr) { + mvccCrd = crd; + mvccCntr = cntr; + mvccOpCntr = opCntr; + + // reset tx state + mvccTxState = TxState.NA; + } + + /** + * @param mvccTxState Mvcc version Tx state hint. + */ + public void mvccTxState(byte mvccTxState) { + this.mvccTxState = mvccTxState; + } + + /** + * @param newMvccTxState New Mvcc version Tx state hint. + */ + public void newMvccTxState(byte newMvccTxState) { + this.newMvccTxState = newMvccTxState; + } + + /** {@inheritDoc} */ + @Override public int size() throws IgniteCheckedException { + return super.size() + MVCC_INFO_SIZE; + } + + /** {@inheritDoc} */ + @Override public int headerSize() { + return MVCC_INFO_SIZE; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccDataRow.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java new file mode 100644 index 0000000000000..81b1b58c4497a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.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.processors.cache.tree.mvcc.data; + +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionImpl; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.tree.RowLinkIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.compare; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.isActive; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.isVisible; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccVersionIsValid; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.unexpectedStateException; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_BIT_OFF; +import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.MVCC_HINTS_MASK; + +/** + * + */ +public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult, BPlusTree.TreeVisitorClosure { + /** */ + private static final int FIRST = DIRTY << 1; + /** */ + private static final int CHECK_VERSION = FIRST << 1; + /** */ + private static final int LAST_COMMITTED_FOUND = CHECK_VERSION << 1; + /** */ + private static final int CAN_CLEANUP = LAST_COMMITTED_FOUND << 1; + /** */ + private static final int PRIMARY = CAN_CLEANUP << 1; + /** */ + private static final int REMOVE_OR_LOCK = PRIMARY << 1; + /** */ + private static final int NEED_HISTORY = REMOVE_OR_LOCK << 1; + /** + * During mvcc transaction processing conflicting row version could be met in storage. + * Not all such cases should lead to transaction abort. + * E.g. if UPDATE for a row meets concurrent INSERT for the same row + * (and row did not exist before both operations) then it means that UPDATE does not see the row at all + * and can proceed. + * This flag enables such mode when conflicting version should not lead to abort immediately + * but more versions should be checked. + */ + private static final int FAST_UPDATE = NEED_HISTORY << 1; + /** */ + private static final int FAST_MISMATCH = FAST_UPDATE << 1; + /** */ + private static final int DELETED = FAST_MISMATCH << 1; + + /** */ + @GridToStringExclude + private final GridCacheContext cctx; + + /** */ + private ResultType res; + + /** */ + @GridToStringExclude + private int state; + + /** */ + private List cleanupRows; + + /** */ + private final MvccSnapshot mvccSnapshot; + + /** */ + private CacheDataRow oldRow; + + /** */ + @GridToStringExclude + private long resCrd; + + /** */ + @GridToStringExclude + private long resCntr; + + /** */ + private List historyRows; + + /** + * @param cctx Cache context. + * @param key Key. + * @param val Value. + * @param ver Version. + * @param part Partition. + * @param expireTime Expire time. + * @param mvccSnapshot MVCC snapshot. + * @param newVer Update version. + * @param primary Primary node flag. + * @param lockOnly Whether no actual update should be done and the only thing to do is to acquire lock. + * @param needHistory Whether to collect rows created or affected by the current tx. + * @param fastUpdate Fast update visit mode. + */ + public MvccUpdateDataRow( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + int part, + long expireTime, + MvccSnapshot mvccSnapshot, + MvccVersion newVer, + boolean primary, + boolean lockOnly, + boolean needHistory, + boolean fastUpdate) { + super(key, + val, + ver, + part, + expireTime, + cctx.cacheId(), + mvccSnapshot, + newVer); + + this.mvccSnapshot = mvccSnapshot; + this.cctx = cctx; + + assert !lockOnly || val == null; + + int flags = FIRST; + + if (primary) + flags |= PRIMARY | CHECK_VERSION; + + if (primary && (lockOnly || val == null)) + flags |= CAN_WRITE | REMOVE_OR_LOCK; + + if (needHistory) + flags |= NEED_HISTORY; + + if (fastUpdate) + flags |= FAST_UPDATE; + + setFlags(flags); + } + + /** {@inheritDoc} */ + @Override public int visit(BPlusTree tree, + BPlusIO io, + long pageAddr, + int idx, IgniteWriteAheadLogManager wal) + throws IgniteCheckedException { + unsetFlags(DIRTY); + + RowLinkIO rowIo = (RowLinkIO)io; + + // Check if entry is locked on primary node. + if (isFlagsSet(PRIMARY | FIRST)) { + long lockCrd = rowIo.getMvccLockCoordinatorVersion(pageAddr, idx); + long lockCntr = rowIo.getMvccLockCounter(pageAddr, idx); + + // We cannot continue while entry is locked by another transaction. + if ((lockCrd != mvccCrd || lockCntr != mvccCntr) + && isActive(cctx, lockCrd, lockCntr, mvccSnapshot)) { + resCrd = lockCrd; + resCntr = lockCntr; + + res = ResultType.LOCKED; + + return setFlags(STOP); + } + } + + MvccDataRow row = (MvccDataRow)tree.getRow(io, pageAddr, idx, RowData.LINK_WITH_HEADER); + + // Check whether the row was updated by current transaction. + // In this case the row is already locked by current transaction and visible to it. + if (isFlagsSet(FIRST)) { + boolean removed = row.newMvccCoordinatorVersion() != MVCC_CRD_COUNTER_NA; + + long rowCrd, rowCntr; int rowOpCntr; + + if (removed) { + rowCrd = row.newMvccCoordinatorVersion(); + rowCntr = row.newMvccCounter(); + rowOpCntr = row.newMvccOperationCounter(); + } + else { + rowCrd = row.mvccCoordinatorVersion(); + rowCntr = row.mvccCounter(); + rowOpCntr = row.mvccOperationCounter(); + } + + if (compare(mvccSnapshot, rowCrd, rowCntr) == 0) { + res = mvccOpCntr == rowOpCntr ? ResultType.VERSION_FOUND : + removed ? ResultType.PREV_NULL : ResultType.PREV_NOT_NULL; + + if (removed) + setFlags(DELETED); + else + oldRow = row; + + setFlags(LAST_COMMITTED_FOUND); + } + } + + long rowLink = row.link(); + + long rowCrd = row.mvccCoordinatorVersion(); + long rowCntr = row.mvccCounter(); + + // with hint bits + int rowOpCntr = (row.mvccTxState() << MVCC_HINTS_BIT_OFF) | (row.mvccOperationCounter() & ~MVCC_HINTS_MASK); + + long rowNewCrd = row.newMvccCoordinatorVersion(); + long rowNewCntr = row.newMvccCounter(); + + // with hint bits + int rowNewOpCntr = (row.newMvccTxState() << MVCC_HINTS_BIT_OFF) | (row.newMvccOperationCounter() & ~MVCC_HINTS_MASK); + + // Search for youngest committed by another transaction row. + if (!isFlagsSet(LAST_COMMITTED_FOUND)) { + if (!(resCrd == rowCrd && resCntr == rowCntr)) { // It's possible it is a chain of aborted changes + byte txState = MvccUtils.state(cctx, rowCrd, rowCntr, rowOpCntr); + + if (txState == TxState.COMMITTED) { + setFlags(LAST_COMMITTED_FOUND); + + if (rowNewCrd != MVCC_CRD_COUNTER_NA) { + if (rowNewCrd == rowCrd && rowNewCntr == rowCntr) + // Row was deleted by the same Tx it was created + txState = TxState.COMMITTED; + else if (rowNewCrd == resCrd && rowNewCntr == resCntr) + // The row is linked to the previously checked aborted version; + txState = TxState.ABORTED; + else + // Check with TxLog if removed version is committed; + txState = MvccUtils.state(cctx, rowNewCrd, rowNewCntr, rowNewOpCntr); + + if (!(txState == TxState.COMMITTED || txState == TxState.ABORTED)) + throw unexpectedStateException(cctx, txState, rowNewCrd, rowNewCntr, rowNewOpCntr, mvccSnapshot); + + if (txState == TxState.COMMITTED) + setFlags(DELETED); + } + + if (isFlagsSet(DELETED)) + res = ResultType.PREV_NULL; + else { + res = ResultType.PREV_NOT_NULL; + + oldRow = row; + } + + if (isFlagsSet(CHECK_VERSION)) { + long crdVer, cntr; int opCntr; + + if (isFlagsSet(DELETED)) { + crdVer = rowNewCrd; + cntr = rowNewCntr; + opCntr = rowNewOpCntr; + } + else { + crdVer = rowCrd; + cntr = rowCntr; + opCntr = rowOpCntr; + } + + // If last committed row is not visible it is possible write conflict. + if (!isVisible(cctx, mvccSnapshot, crdVer, cntr, opCntr, false)) { + // In case when row is accessed without previous version check (FAST_UPDATE) + // it is possible that we should consider this row non existent for current transaction + // without signalling write conflict. + // To do this we need to find youngest visible version and if it is removed version + // or there is no visible version then there is no conflict. + if (isFlagsSet(FAST_UPDATE) + && !(isFlagsSet(DELETED) + && isVisible(cctx, mvccSnapshot, rowCrd, rowCntr, rowOpCntr, false))) { + res = ResultType.PREV_NULL; + + setFlags(FAST_MISMATCH); + } + else { + resCrd = crdVer; + resCntr = cntr; + + res = ResultType.VERSION_MISMATCH; // Write conflict. + + return setFlags(STOP); + } + } + } + + // Lock entry for primary partition if needed. + // If invisible row is found for FAST_UPDATE case we should not lock row. + if (isFlagsSet(PRIMARY | REMOVE_OR_LOCK) && !isFlagsSet(FAST_MISMATCH)) { + rowIo.setMvccLockCoordinatorVersion(pageAddr, idx, mvccCrd); + rowIo.setMvccLockCounter(pageAddr, idx, mvccCntr); + + // TODO Delta record IGNITE-7991 + + setFlags(DIRTY); + } + + unsetFlags(CAN_WRITE); // No need to acquire write locks anymore + } + else if (txState == TxState.ABORTED) { // save aborted version to fast check new version of next row + resCrd = rowCrd; + resCntr = rowCntr; + } + else + throw unexpectedStateException(cctx, txState, rowCrd, rowCntr, rowOpCntr, mvccSnapshot); + } + } + // Search for youngest visible row. + // If we have not found any visible version then we does not see this row. + else if (isFlagsSet(FAST_MISMATCH)) { + assert !isFlagsSet(CAN_CLEANUP); + assert mvccVersionIsValid(rowNewCrd, rowNewCntr, rowNewOpCntr); + + // Update version could be visible only if it is removal version, + // previous create versions were already checked in previous step and are definitely invisible. + // If we found visible removal version then we does not see this row. + if (isVisible(cctx, mvccSnapshot, rowNewCrd, rowNewCntr, rowNewOpCntr, false)) + unsetFlags(FAST_MISMATCH); + // If the youngest visible for current transaction version is not removal version then it is write conflict. + else if (isVisible(cctx, mvccSnapshot, rowCrd, rowCntr, rowOpCntr, false)) { + resCrd = rowCrd; + resCntr = rowCntr; + + res = ResultType.VERSION_MISMATCH; + + return setFlags(STOP); + } + } + + long cleanupVer = mvccSnapshot.cleanupVersion(); + + if (cleanupVer > MVCC_OP_COUNTER_NA // Do not clean if cleanup version is not assigned. + && !isFlagsSet(CAN_CLEANUP) + && isFlagsSet(LAST_COMMITTED_FOUND | DELETED)) { + assert mvccVersionIsValid(rowNewCrd, rowNewCntr, rowNewOpCntr); + + // We can cleanup previous row only if it was deleted by another + // transaction and delete version is less or equal to cleanup one + if (rowNewCrd < mvccCrd || Long.compare(cleanupVer, rowNewCntr) >= 0) + setFlags(CAN_CLEANUP); + } + + if (isFlagsSet(CAN_CLEANUP) + || !isFlagsSet(LAST_COMMITTED_FOUND)) { // can cleanup aborted versions + if (cleanupRows == null) + cleanupRows = new ArrayList<>(); + + cleanupRows.add(new MvccLinkAwareSearchRow(cacheId, key, rowCrd, rowCntr, rowOpCntr & ~MVCC_HINTS_MASK, rowLink)); + } + else { + // Row obsoleted by current operation, all rows created or updated with current tx. + if (isFlagsSet(NEED_HISTORY) + && (row == oldRow + || (rowCrd == mvccCrd && rowCntr == mvccCntr) + || (rowNewCrd == mvccCrd && rowNewCntr == mvccCntr))) { + if (historyRows == null) + historyRows = new ArrayList<>(); + + historyRows.add(new MvccLinkAwareSearchRow(cacheId, key, rowCrd, rowCntr, rowOpCntr & ~MVCC_HINTS_MASK, rowLink)); + } + + if (cleanupVer > MVCC_OP_COUNTER_NA // Do not clean if cleanup version is not assigned. + && !isFlagsSet(CAN_CLEANUP) + && isFlagsSet(LAST_COMMITTED_FOUND) + && (rowCrd < mvccCrd || Long.compare(cleanupVer, rowCntr) >= 0)) + // all further versions are guaranteed to be less than cleanup version + setFlags(CAN_CLEANUP); + } + + return unsetFlags(FIRST); + } + + /** {@inheritDoc} */ + @Override public int state() { + return state; + } + + /** + * @return Old row. + */ + public CacheDataRow oldRow() { + return oldRow; + } + + /** + * @return {@code True} if previous value was non-null. + */ + @Override public ResultType resultType() { + return res == null ? ResultType.PREV_NULL : res; + } + + /** + * @return Rows which are safe to cleanup. + */ + public List cleanupRows() { + return cleanupRows; + } + + /** + * @return Result version. + */ + @Override public MvccVersion resultVersion() { + switch (resultType()) { + case VERSION_FOUND: + case PREV_NULL: + + return new MvccVersionImpl(mvccCrd, mvccCntr, mvccOpCntr); + case PREV_NOT_NULL: + + return new MvccVersionImpl(oldRow.mvccCoordinatorVersion(), oldRow.mvccCounter(), oldRow.mvccOperationCounter()); + case LOCKED: + case VERSION_MISMATCH: + + assert resCrd != MVCC_CRD_COUNTER_NA && resCntr != MVCC_COUNTER_NA; + + return new MvccVersionImpl(resCrd, resCntr, MVCC_OP_COUNTER_NA); + default: + + throw new IllegalStateException("Unexpected result type: " + resultType()); + } + } + + /** {@inheritDoc} */ + @Override public List history() { + if (isFlagsSet(NEED_HISTORY) && historyRows == null) + historyRows = new ArrayList<>(); + + return historyRows; + } + + /** */ + private boolean isFlagsSet(int flags) { + return (state & flags) == flags; + } + + /** */ + private int setFlags(int flags) { + return state |= flags; + } + + /** */ + private int unsetFlags(int flags) { + return state &= (~flags); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccUpdateDataRow.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRowNative.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRowNative.java new file mode 100644 index 0000000000000..38611a9572a11 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRowNative.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.tree.mvcc.data; + +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.tree.RowLinkIO; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class MvccUpdateDataRowNative extends MvccDataRow implements BPlusTree.TreeRowClosure { + /** */ + private final MvccSnapshot mvccSnapshot; + /** */ + private ResultType res; + /** */ + private boolean canCleanup; + /** */ + private GridLongList activeTxs; + /** */ + private List cleanupRows; + /** */ + private CacheDataRow oldRow; + + /** + * @param key Key. + * @param val Value. + * @param ver Version. + * @param expireTime Expire time. + * @param mvccSnapshot MVCC snapshot. + * @param newVer Update version. + * @param part Partition. + * @param cctx Cache context. + */ + public MvccUpdateDataRowNative( + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot, + MvccVersion newVer, + int part, + GridCacheContext cctx) { + super(key, + val, + ver, + part, + expireTime, + cctx.cacheId(), + mvccSnapshot, + newVer); + + this.mvccSnapshot = mvccSnapshot; + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, + BPlusIO io, + long pageAddr, + int idx) + throws IgniteCheckedException { + RowLinkIO rowIo = (RowLinkIO)io; + + // Assert version grows. + assert assertVersion(rowIo, pageAddr, idx); + + boolean checkActive = mvccSnapshot.activeTransactions().size() > 0; + + boolean txActive = false; + + long rowCrdVer = rowIo.getMvccCoordinatorVersion(pageAddr, idx); + + long crdVer = mvccCoordinatorVersion(); + + boolean isFirstRmvd = false; + + if (res == null) { + int cmp = Long.compare(crdVer, rowCrdVer); + + if (cmp == 0) + cmp = Long.compare(mvccSnapshot.counter(), rowIo.getMvccCounter(pageAddr, idx)); + + if (cmp == 0) + res = ResultType.VERSION_FOUND; + else { + oldRow = tree.getRow(io, pageAddr, idx, RowData.LINK_WITH_HEADER); + + isFirstRmvd = oldRow.newMvccCoordinatorVersion() != 0; + + if (isFirstRmvd) + res = ResultType.PREV_NULL; + else + res = ResultType.PREV_NOT_NULL; + } + } + + // Suppose transactions on previous coordinator versions are done. + if (checkActive && crdVer == rowCrdVer) { + long rowMvccCntr = rowIo.getMvccCounter(pageAddr, idx); + + long activeTx = isFirstRmvd ? oldRow.newMvccCounter() : rowMvccCntr; + + if (mvccSnapshot.activeTransactions().contains(activeTx)) { + txActive = true; + + if (activeTxs == null) + activeTxs = new GridLongList(); + + activeTxs.add(activeTx); + } + } + + if (!txActive) { + assert Long.compare(crdVer, rowCrdVer) >= 0; + + int cmp; + + long rowCntr = rowIo.getMvccCounter(pageAddr, idx); + + if (crdVer == rowCrdVer) + cmp = Long.compare(mvccSnapshot.cleanupVersion(), rowCntr); + else + cmp = 1; + + if (cmp >= 0) { + // Do not cleanup oldest version. + if (canCleanup) { + assert MvccUtils.mvccVersionIsValid(rowCrdVer, rowCntr); + + // Should not be possible to cleanup active tx. + assert rowCrdVer != crdVer || !mvccSnapshot.activeTransactions().contains(rowCntr); + + if (cleanupRows == null) + cleanupRows = new ArrayList<>(); + + cleanupRows.add(new MvccLinkAwareSearchRow(cacheId, key, rowCrdVer, rowCntr, + rowIo.getMvccOperationCounter(pageAddr, idx), rowIo.getLink(pageAddr, idx))); + } + else + canCleanup = true; + } + } + + return true; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return MvccUtils.MVCC_START_OP_CNTR; + } + + /** + * @return Old row. + */ + public CacheDataRow oldRow() { + return oldRow; + } + + /** + * @return {@code True} if previous value was non-null. + */ + public ResultType resultType() { + return res == null ? ResultType.PREV_NULL : res; + } + + /** + * @return Active transactions to wait for. + */ + @Nullable public GridLongList activeTransactions() { + return activeTxs; + } + + /** + * @return Rows which are safe to cleanup. + */ + public List cleanupRows() { + return cleanupRows; + } + + /** + * @param io IO. + * @param pageAddr Page address. + * @param idx Item index. + * @return Always {@code true}. + */ + private boolean assertVersion(RowLinkIO io, long pageAddr, int idx) { + long rowCrdVer = io.getMvccCoordinatorVersion(pageAddr, idx); + long rowCntr = io.getMvccCounter(pageAddr, idx); + + int cmp = Long.compare(mvccCoordinatorVersion(), rowCrdVer); + + if (cmp == 0) + cmp = Long.compare(mvccSnapshot.counter(), rowCntr); + + // Can be equals if execute update on backup and backup already rebalanced value updated on primary. + assert cmp >= 0 : "[updCrd=" + mvccCoordinatorVersion() + + ", updCntr=" + mvccSnapshot.counter() + + ", rowCrd=" + rowCrdVer + + ", rowCntr=" + rowCntr + ']'; + + return true; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccUpdateDataRowNative.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java new file mode 100644 index 0000000000000..086a30df9c0b4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.tree.mvcc.data; + +import java.util.List; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; + +/** + * + */ +public interface MvccUpdateResult { + /** + * @return Type of result. + */ + public ResultType resultType(); + + /** + * @return Result version. + */ + public MvccVersion resultVersion(); + + /** + * + * @return Collection of row created or affected by the current tx. + */ + public List history(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/ResultType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/ResultType.java new file mode 100644 index 0000000000000..eecb4a548f1e1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/ResultType.java @@ -0,0 +1,34 @@ +/* + * 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.tree.mvcc.data; + +/** + * + */ +public enum ResultType { + /** */ + VERSION_FOUND, + /** */ + PREV_NULL, + /** */ + PREV_NOT_NULL, + /** */ + LOCKED, + /** */ + VERSION_MISMATCH +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccFirstRowTreeClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccFirstRowTreeClosure.java new file mode 100644 index 0000000000000..8546a66dca76c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccFirstRowTreeClosure.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.processors.cache.tree.mvcc.search; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.tree.RowLinkIO; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.getNewVersion; + +/** + * Closure which returns the very first encountered row. + */ +public class MvccFirstRowTreeClosure implements MvccTreeClosure { + /** */ + private final GridCacheContext cctx; + + /** */ + private CacheDataRow res; + + /** + * @param cctx Cache context. + */ + public MvccFirstRowTreeClosure(GridCacheContext cctx) { + this.cctx = cctx; + } + + /** + * @return Found row. + */ + @Nullable public CacheDataRow row() { + return res; + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, + long pageAddr, int idx) throws IgniteCheckedException { + RowLinkIO rowIo = (RowLinkIO)io; + + MvccVersion newVersion = getNewVersion(cctx, rowIo.getLink(pageAddr, idx)); + + if (newVersion == null) + res = tree.getRow(io, pageAddr, idx, CacheDataRowAdapter.RowData.NO_KEY); + + return false; // Stop search. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccFirstRowTreeClosure.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccLinkAwareSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccLinkAwareSearchRow.java new file mode 100644 index 0000000000000..1e763a6a2d6ae --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccLinkAwareSearchRow.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.tree.mvcc.search; + +import org.apache.ignite.internal.processors.cache.KeyCacheObject; + +/** + * MVCC search row which contains a link. Now used only for cleanup purposes. + */ +public class MvccLinkAwareSearchRow extends MvccSearchRow { + /** */ + private final long link; + + /** + * @param cacheId Cache ID. + * @param key Key. + * @param crdVer Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @param link Link. + */ + public MvccLinkAwareSearchRow(int cacheId, KeyCacheObject key, long crdVer, long mvccCntr, int mvccOpCntr, long link) { + super(cacheId, key, crdVer, mvccCntr, mvccOpCntr); + + assert link != 0L; + + this.link = link; + } + + /** {@inheritDoc} */ + @Override public long link() { + return link; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMaxSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMaxSearchRow.java new file mode 100644 index 0000000000000..f5f3b67afaf12 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMaxSearchRow.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.tree.mvcc.search; + +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.tree.SearchRow; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Search row for maximum key version. + */ +public class MvccMaxSearchRow extends SearchRow { + /** + * @param cacheId Cache ID. + * @param key Key. + */ + public MvccMaxSearchRow(int cacheId, KeyCacheObject key) { + super(cacheId, key); + } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return Long.MAX_VALUE; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return Long.MAX_VALUE; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return Integer.MAX_VALUE; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccMaxSearchRow.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMinSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMinSearchRow.java new file mode 100644 index 0000000000000..a1cb8fcf51056 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccMinSearchRow.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.tree.mvcc.search; + +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.tree.SearchRow; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Search row for minimum key version. + */ +public class MvccMinSearchRow extends SearchRow { + /** + * @param cacheId Cache ID. + * @param key Key. + */ + public MvccMinSearchRow(int cacheId, KeyCacheObject key) { + super(cacheId, key); + } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return 1L; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return 1L; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return 1; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccMinSearchRow.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSearchRow.java new file mode 100644 index 0000000000000..d0510f764b262 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSearchRow.java @@ -0,0 +1,70 @@ +/* + * 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.tree.mvcc.search; + +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.tree.SearchRow; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * MVCC search row which uses specific MVCC version (coordinator + counter) to filter rows. + */ +public class MvccSearchRow extends SearchRow { + /** Coordinator version. */ + protected long crdVer; + + /** Counter. */ + private long cntr; + + /** Operation counter. */ + protected int opCntr; + + /** + * @param cacheId Cache ID. + * @param key Key. + * @param crdVer Coordinator version. + * @param cntr Counter. + */ + public MvccSearchRow(int cacheId, KeyCacheObject key, long crdVer, long cntr, int opCntr) { + super(cacheId, key); + + this.crdVer = crdVer; + this.cntr = cntr; + this.opCntr = opCntr; + } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return crdVer; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return cntr; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return opCntr; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccSearchRow.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSnapshotSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSnapshotSearchRow.java new file mode 100644 index 0000000000000..432593d0f0044 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccSnapshotSearchRow.java @@ -0,0 +1,113 @@ +/* + * 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.tree.mvcc.search; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.tree.RowLinkIO; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_INVISIBLE; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_VISIBLE; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_VISIBLE_REMOVED; + +/** + * Search row which returns the first row visible for the given snapshot. Usage: + * - set this row as the upper bound + * - pass the same row as search closure. + */ +public class MvccSnapshotSearchRow extends MvccSearchRow implements MvccTreeClosure { + /** */ + private final GridCacheContext cctx; + + /** Resulting row. */ + private CacheDataRow res; + + /** */ + private MvccSnapshot snapshot; + + /** + * Constructor. + * + * @param cctx + * @param key Key. + * @param snapshot Snapshot. + */ + public MvccSnapshotSearchRow(GridCacheContext cctx, KeyCacheObject key, + MvccSnapshot snapshot) { + super(cctx.cacheId(), key, snapshot.coordinatorVersion(), snapshot.counter(), Integer.MAX_VALUE); + + this.cctx = cctx; + + this.snapshot = snapshot; + } + + /** + * @return Found row. + */ + @Nullable public CacheDataRow row() { + return res; + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, + long pageAddr, int idx) throws IgniteCheckedException { + RowLinkIO rowIo = (RowLinkIO)io; + + long rowCrdVer = rowIo.getMvccCoordinatorVersion(pageAddr, idx); + long rowCntr = rowIo.getMvccCounter(pageAddr, idx); + int rowOpCntr = rowIo.getMvccOperationCounter(pageAddr, idx); + + assert MvccUtils.mvccVersionIsValid(rowCrdVer, rowCntr, rowOpCntr); + + if (MvccUtils.isVisible(cctx, snapshot, rowCrdVer, rowCntr, rowOpCntr, false)) { + int state = MvccUtils.getVisibleState(cctx, rowIo.getLink(pageAddr, idx), snapshot); + + if (state == MVCC_INVISIBLE) + return true; + + if (state == MVCC_VISIBLE_REMOVED) + res = null; + else { + assert state == MVCC_VISIBLE; + + res = tree.getRow(io, pageAddr, idx, CacheDataRowAdapter.RowData.NO_KEY); + + res.key(key()); + } + + return false; // Stop search. + } + + return true; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccSnapshotSearchRow.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccTreeClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccTreeClosure.java new file mode 100644 index 0000000000000..031903f5ae10a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/search/MvccTreeClosure.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.internal.processors.cache.tree.mvcc.search; + +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; + +/** + * Common interface for MVCC search row closures (no-op, only to have clear inheritance hierarchy). + */ +public interface MvccTreeClosure extends BPlusTree.TreeRowClosure { + // No-op. +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java index eff680f48b718..68ac200f428f6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlStateCode.java @@ -61,6 +61,9 @@ private SqlStateCode() { /** Requested operation is not supported. */ public final static String UNSUPPORTED_OPERATION = "0A000"; + /** Transaction state exception. */ + public final static String TRANSACTION_STATE_EXCEPTION = "25000"; + /** Parsing exception. */ public final static String PARSING_EXCEPTION = "42000"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java index bdc558c0daad7..065a29880c8ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java @@ -39,6 +39,9 @@ public class JdbcBatchExecuteRequest extends JdbcRequest { @GridToStringInclude(sensitive = true) private List queries; + /** Client auto commit flag state. */ + private boolean autoCommit; + /** * Last stream batch flag - whether open streamers on current connection * must be flushed and closed after this batch. @@ -63,15 +66,17 @@ protected JdbcBatchExecuteRequest(byte type) { /** * @param schemaName Schema name. * @param queries Queries. + * @param autoCommit Client auto commit flag state. * @param lastStreamBatch {@code true} in case the request is the last batch at the stream. */ - public JdbcBatchExecuteRequest(String schemaName, List queries, boolean lastStreamBatch) { + public JdbcBatchExecuteRequest(String schemaName, List queries, boolean autoCommit, boolean lastStreamBatch) { super(BATCH_EXEC); assert lastStreamBatch || !F.isEmpty(queries); this.schemaName = schemaName; this.queries = queries; + this.autoCommit = autoCommit; this.lastStreamBatch = lastStreamBatch; } @@ -81,15 +86,17 @@ public JdbcBatchExecuteRequest(String schemaName, List queries, boole * @param type Request type. * @param schemaName Schema name. * @param queries Queries. + * @param autoCommit Client auto commit flag state. * @param lastStreamBatch {@code true} in case the request is the last batch at the stream. */ - protected JdbcBatchExecuteRequest(byte type, String schemaName, List queries, boolean lastStreamBatch) { + protected JdbcBatchExecuteRequest(byte type, String schemaName, List queries, boolean autoCommit, boolean lastStreamBatch) { super(type); assert lastStreamBatch || !F.isEmpty(queries); this.schemaName = schemaName; this.queries = queries; + this.autoCommit = autoCommit; this.lastStreamBatch = lastStreamBatch; } @@ -107,6 +114,13 @@ public List queries() { return queries; } + /** + * @return Auto commit flag. + */ + boolean autoCommit() { + return autoCommit; + } + /** * @return Last stream batch flag. */ @@ -125,14 +139,17 @@ public boolean isLastStreamBatch() { for (JdbcQuery q : queries) q.writeBinary(writer); + } else writer.writeInt(0); + writer.writeBoolean(autoCommit); writer.writeBoolean(lastStreamBatch); } /** {@inheritDoc} */ + @SuppressWarnings("SimplifiableIfStatement") @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { super.readBinary(reader); @@ -151,8 +168,12 @@ public boolean isLastStreamBatch() { } try { - if (reader.available() > 0) + if (reader.available() > 0) { + autoCommit = reader.readBoolean(); lastStreamBatch = reader.readBoolean(); + } + else + autoCommit = true; } catch (IOException e) { throw new BinaryObjectException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java index 23f4d498d45ac..9f3fe2a6277bc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java @@ -29,8 +29,10 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; +import org.apache.ignite.internal.processors.query.NestedTxMode; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.nio.GridNioSession; +import org.apache.ignite.internal.util.typedef.F; /** * JDBC Connection Context. @@ -129,29 +131,44 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin boolean autoCloseCursors = reader.readBoolean(); boolean lazyExec = false; + boolean skipReducerOnUpdate = false; + + NestedTxMode nestedTxMode = NestedTxMode.DEFAULT; + AuthorizationContext actx = null; if (ver.compareTo(VER_2_1_5) >= 0) lazyExec = reader.readBoolean(); - boolean skipReducerOnUpdate = false; - if (ver.compareTo(VER_2_3_0) >= 0) skipReducerOnUpdate = reader.readBoolean(); - String user = null; - String passwd = null; + if (ver.compareTo(VER_2_5_0) >= 0) { + String nestedTxModeName = reader.readString(); - try { - if (reader.available() > 0) { - user = reader.readString(); - passwd = reader.readString(); + if (!F.isEmpty(nestedTxModeName)) { + try { + nestedTxMode = NestedTxMode.valueOf(nestedTxModeName); + } + catch (IllegalArgumentException e) { + throw new IgniteCheckedException("Invalid nested transactions handling mode: " + nestedTxModeName); + } } - } - catch (Exception e) { - throw new IgniteCheckedException("Handshake error: " + e.getMessage(), e); - } - AuthorizationContext actx = authenticate(user, passwd); + String user = null; + String passwd = null; + + try { + if (reader.available() > 0) { + user = reader.readString(); + passwd = reader.readString(); + } + } + catch (Exception e) { + throw new IgniteCheckedException("Handshake error: " + e.getMessage(), e); + } + + actx = authenticate(user, passwd); + } parser = new JdbcMessageParser(ctx); @@ -169,7 +186,9 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin }; handler = new JdbcRequestHandler(ctx, busyLock, sender, maxCursors, distributedJoins, enforceJoinOrder, - collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, actx, ver); + collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, nestedTxMode, actx, ver); + + handler.start(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java index 3e847310af232..1d4d4ac4838f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java @@ -42,12 +42,13 @@ public JdbcOrderedBatchExecuteRequest() { /** * @param schemaName Schema name. * @param queries Queries. + * @param autoCommit Client auto commit flag state. * @param lastStreamBatch {@code true} in case the request is the last batch at the stream. * @param order Request order. */ public JdbcOrderedBatchExecuteRequest(String schemaName, List queries, - boolean lastStreamBatch, long order) { - super(BATCH_EXEC_ORDERED, schemaName, queries, lastStreamBatch); + boolean autoCommit, boolean lastStreamBatch, long order) { + super(BATCH_EXEC_ORDERED, schemaName, queries, autoCommit, lastStreamBatch); this.order = order; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java index 3e54fc8515615..c13117a2dfbac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java @@ -51,6 +51,9 @@ public class JdbcQueryExecuteRequest extends JdbcRequest { /** Expected statement type. */ private JdbcStatementType stmtType; + /** Client auto commit flag state. */ + private boolean autoCommit; + /** */ JdbcQueryExecuteRequest() { @@ -62,11 +65,12 @@ public class JdbcQueryExecuteRequest extends JdbcRequest { * @param schemaName Cache name. * @param pageSize Fetch size. * @param maxRows Max rows. + * @param autoCommit Connection auto commit flag state. * @param sqlQry SQL query. * @param args Arguments list. */ public JdbcQueryExecuteRequest(JdbcStatementType stmtType, String schemaName, int pageSize, int maxRows, - String sqlQry, Object[] args) { + boolean autoCommit, String sqlQry, Object[] args) { super(QRY_EXEC); this.schemaName = F.isEmpty(schemaName) ? null : schemaName; @@ -75,6 +79,7 @@ public JdbcQueryExecuteRequest(JdbcStatementType stmtType, String schemaName, in this.sqlQry = sqlQry; this.args = args; this.stmtType = stmtType; + this.autoCommit = autoCommit; } /** @@ -119,6 +124,13 @@ public JdbcStatementType expectedStatementType() { return stmtType; } + /** + * @return Auto commit flag. + */ + boolean autoCommit() { + return autoCommit; + } + /** {@inheritDoc} */ @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { super.writeBinary(writer); @@ -136,9 +148,12 @@ public JdbcStatementType expectedStatementType() { } writer.writeByte((byte)stmtType.ordinal()); + + writer.writeBoolean(autoCommit); } /** {@inheritDoc} */ + @SuppressWarnings("SimplifiableIfStatement") @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { super.readBinary(reader); @@ -163,6 +178,16 @@ public JdbcStatementType expectedStatementType() { catch (IOException e) { throw new BinaryObjectException(e); } + + try { + if (reader.available() > 0) + autoCommit = reader.readBoolean(); + else + autoCommit = true; + } + catch (IOException e) { + throw new BinaryObjectException(e); + } } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java index 8facd3270fcbb..7dda67fc926ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java @@ -33,10 +33,10 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import javax.cache.configuration.Factory; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.BulkLoadContextCursor; import org.apache.ignite.cache.query.FieldsQueryCursor; -import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteVersionUtils; @@ -56,9 +56,11 @@ import org.apache.ignite.internal.processors.query.GridQueryProperty; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.processors.query.NestedTxMode; import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.processors.query.SqlClientContext; import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.S; @@ -105,6 +107,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Busy lock. */ private final GridSpinBusyLock busyLock; + /** Worker. */ + private final JdbcRequestHandlerWorker worker; + /** Maximum allowed cursors. */ private final int maxCursors; @@ -126,6 +131,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Automatic close of cursors. */ private final boolean autoCloseCursors; + /** Nested transactions handling mode. */ + private final NestedTxMode nestedTxMode; + /** Protocol version. */ private ClientListenerProtocolVersion protocolVer; @@ -151,7 +159,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, JdbcResponseSender sender, int maxCursors, boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly, - boolean autoCloseCursors, boolean lazy, boolean skipReducerOnUpdate, + boolean autoCloseCursors, boolean lazy, boolean skipReducerOnUpdate, NestedTxMode nestedTxMode, AuthorizationContext actx, ClientListenerProtocolVersion protocolVer) { this.ctx = ctx; this.sender = sender; @@ -176,10 +184,16 @@ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, this.busyLock = busyLock; this.maxCursors = maxCursors; this.autoCloseCursors = autoCloseCursors; + this.nestedTxMode = nestedTxMode; this.protocolVer = protocolVer; this.actx = actx; log = ctx.log(getClass()); + + if (ctx.grid().configuration().isMvccEnabled()) + worker = new JdbcRequestHandlerWorker(ctx.igniteInstanceName(), log, this, ctx); + else + worker = null; } /** {@inheritDoc} */ @@ -190,6 +204,34 @@ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, JdbcRequest req = (JdbcRequest)req0; + if (worker == null) + return doHandle(req); + else { + GridFutureAdapter fut = worker.process(req); + + try { + return fut.get(); + } + catch (IgniteCheckedException e) { + return exceptionToResult(e); + } + } + } + + /** + * Start worker, if it's present. + */ + void start() { + if (worker != null) + worker.start(); + } + + /** + * Actually handle the request. + * @param req Request. + * @return Request handling result. + */ + ClientListenerResponse doHandle(JdbcRequest req) { if (!busyLock.enterBusy()) return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Failed to handle JDBC request because node is stopping."); @@ -364,6 +406,17 @@ private ClientListenerResponse processBulkLoadFileBatch(JdbcBulkLoadBatchRequest public void onDisconnect() { if (busyLock.enterBusy()) { + if (worker != null) { + worker.cancel(); + + try { + worker.join(); + } + catch (InterruptedException e) { + // No-op. + } + } + try { for (JdbcQueryCursor cursor : qryCursors.values()) @@ -411,11 +464,11 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { try { String sql = req.sqlQuery(); - SqlFieldsQuery qry; + SqlFieldsQueryEx qry; switch(req.expectedStatementType()) { case ANY_STATEMENT_TYPE: - qry = new SqlFieldsQuery(sql); + qry = new SqlFieldsQueryEx(sql, null); break; @@ -440,6 +493,8 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { qry.setCollocated(cliCtx.isCollocated()); qry.setReplicatedOnly(cliCtx.isReplicatedOnly()); qry.setLazy(cliCtx.isLazy()); + qry.setNestedTxMode(nestedTxMode); + qry.setAutoCommit(req.autoCommit()); if (req.pageSize() <= 0) return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Invalid fetch size: " + req.pageSize()); @@ -656,6 +711,8 @@ private ClientListenerResponse executeBatch(JdbcBatchExecuteRequest req) { qry.setCollocated(cliCtx.isCollocated()); qry.setReplicatedOnly(cliCtx.isReplicatedOnly()); qry.setLazy(cliCtx.isLazy()); + qry.setNestedTxMode(nestedTxMode); + qry.setAutoCommit(req.autoCommit()); qry.setSchema(schemaName); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandlerWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandlerWorker.java new file mode 100644 index 0000000000000..72117874512b7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandlerWorker.java @@ -0,0 +1,131 @@ +/* + * 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.jdbc; + +import java.util.concurrent.LinkedBlockingQueue; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.odbc.ClientListenerNioListener; +import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.thread.IgniteThread; +import org.jetbrains.annotations.Nullable; + +/** + * JDBC request handler worker to maintain single threaded transactional execution of SQL statements when MVCC is on.

    + * This worker is intended for internal use as a temporary solution and from within {@link JdbcRequestHandler}, + * therefore it does not do any fine-grained lifecycle handling as it relies on existing guarantees from + * {@link ClientListenerNioListener}. + */ +class JdbcRequestHandlerWorker extends GridWorker { + /** Requests queue.*/ + private final LinkedBlockingQueue>> queue = + new LinkedBlockingQueue<>(); + + /** Handler.*/ + private final JdbcRequestHandler hnd; + + /** Context.*/ + private final GridKernalContext ctx; + + /** Response */ + private final static ClientListenerResponse ERR_RESPONSE = new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, + "Connection closed."); + + /** + * Constructor. + * @param igniteInstanceName Instance name. + * @param log Logger. + * @param hnd Handler. + * @param ctx Kernal context. + */ + JdbcRequestHandlerWorker(@Nullable String igniteInstanceName, IgniteLogger log, JdbcRequestHandler hnd, + GridKernalContext ctx) { + super(igniteInstanceName, "jdbc-request-handler-worker", log); + + A.notNull(hnd, "hnd"); + + this.hnd = hnd; + + this.ctx = ctx; + } + + /** + * Start this worker. + */ + void start() { + new IgniteThread(this).start(); + } + + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + try { + while (!isCancelled()) { + T2> req = queue.take(); + + GridFutureAdapter fut = req.get2(); + + try { + ClientListenerResponse res = hnd.doHandle(req.get1()); + + fut.onDone(res); + } + catch (Exception e) { + fut.onDone(e); + } + } + } + finally { + // Notify indexing that this worker is being stopped. + try { + ctx.query().getIndexing().onClientDisconnect(); + } + catch (Exception e) { + // No-op. + } + + // Drain the queue on stop. + T2> req = queue.poll(); + + while (req != null) { + req.get2().onDone(ERR_RESPONSE); + + req = queue.poll(); + } + } + } + + /** + * Initiate request processing. + * @param req Request. + * @return Future to track request processing. + */ + GridFutureAdapter process(JdbcRequest req) { + GridFutureAdapter fut = new GridFutureAdapter<>(); + + queue.add(new T2<>(req, fut)); + + return fut; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java index 253b183faac92..2b51741d9e93f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.processors.odbc.odbc; -import java.util.HashSet; -import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryReaderExImpl; @@ -27,8 +25,12 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser; import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler; +import org.apache.ignite.internal.processors.query.NestedTxMode; import org.apache.ignite.internal.util.GridSpinBusyLock; +import java.util.HashSet; +import java.util.Set; + /** * ODBC Connection Context. */ @@ -45,7 +47,7 @@ public class OdbcConnectionContext extends ClientListenerAbstractConnectionConte /** Version 2.3.2: added multiple statements support. */ public static final ClientListenerProtocolVersion VER_2_3_2 = ClientListenerProtocolVersion.create(2, 3, 2); - /** Version 2.5.0: added authentication. */ + /** Version 2.5.0: added authentication and transactions. */ public static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0); /** Version 2.7.0: added precision and scale. */ @@ -111,6 +113,7 @@ public OdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, l boolean enforceJoinOrder = reader.readBoolean(); boolean replicatedOnly = reader.readBoolean(); boolean collocated = reader.readBoolean(); + boolean lazy = false; if (ver.compareTo(VER_2_1_5) >= 0) @@ -124,17 +127,25 @@ public OdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, l String user = null; String passwd = null; + NestedTxMode nestedTxMode = NestedTxMode.DEFAULT; + if (ver.compareTo(VER_2_5_0) >= 0) { user = reader.readString(); passwd = reader.readString(); + + byte nestedTxModeVal = reader.readByte(); + + nestedTxMode = NestedTxMode.fromByte(nestedTxModeVal); } AuthorizationContext actx = authenticate(user, passwd); - handler = new OdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, - enforceJoinOrder, replicatedOnly, collocated, lazy, skipReducerOnUpdate, actx, ver); + handler = new OdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder, + replicatedOnly, collocated, lazy, skipReducerOnUpdate, actx, nestedTxMode, ver); parser = new OdbcMessageParser(ctx, ver); + + handler.start(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java index c2137bd469e48..552841d20ddf9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java @@ -47,7 +47,7 @@ public class OdbcMessageParser implements ClientListenerMessageParser { protected static final int INIT_CAP = 1024; /** Kernal context. */ - protected GridKernalContext ctx; + protected final GridKernalContext ctx; /** Logger. */ private final IgniteLogger log; @@ -101,7 +101,12 @@ public OdbcMessageParser(GridKernalContext ctx, ClientListenerProtocolVersion ve if (ver.compareTo(OdbcConnectionContext.VER_2_3_2) >= 0) timeout = reader.readInt(); - res = new OdbcQueryExecuteRequest(schema, sql, params, timeout); + boolean autoCommit = true; + + if (ver.compareTo(OdbcConnectionContext.VER_2_5_0) >= 0) + autoCommit = reader.readBoolean(); + + res = new OdbcQueryExecuteRequest(schema, sql, params, timeout, autoCommit); break; } @@ -123,7 +128,12 @@ public OdbcMessageParser(GridKernalContext ctx, ClientListenerProtocolVersion ve if (ver.compareTo(OdbcConnectionContext.VER_2_3_2) >= 0) timeout = reader.readInt(); - res = new OdbcQueryExecuteBatchRequest(schema, sql, last, params, timeout); + boolean autoCommit = true; + + if (ver.compareTo(OdbcConnectionContext.VER_2_5_0) >= 0) + autoCommit = reader.readBoolean(); + + res = new OdbcQueryExecuteBatchRequest(schema, sql, last, params, timeout, autoCommit); break; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchRequest.java index 0e4effdaf2800..75c2831750905 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteBatchRequest.java @@ -41,6 +41,10 @@ public class OdbcQueryExecuteBatchRequest extends OdbcRequest { @GridToStringExclude private final Object[][] args; + /** Autocommit flag. */ + @GridToStringInclude + private final boolean autoCommit; + /** Query timeout in seconds. */ @GridToStringInclude private final int timeout; @@ -51,9 +55,10 @@ public class OdbcQueryExecuteBatchRequest extends OdbcRequest { * @param last Last page flag. * @param args Arguments list. * @param timeout Timeout in seconds. + * @param autoCommit Autocommit flag. */ public OdbcQueryExecuteBatchRequest(@Nullable String schema, String sqlQry, boolean last, Object[][] args, - int timeout) { + int timeout, boolean autoCommit) { super(QRY_EXEC_BATCH); assert sqlQry != null : "SQL query should not be null"; @@ -64,6 +69,7 @@ public OdbcQueryExecuteBatchRequest(@Nullable String schema, String sqlQry, bool this.last = last; this.args = args; this.timeout = timeout; + this.autoCommit = autoCommit; } /** @@ -106,4 +112,11 @@ public int timeout() { @Override public String toString() { return S.toString(OdbcQueryExecuteBatchRequest.class, this, "args", args, true); } + + /** + * @return Autocommit flag. + */ + public boolean autoCommit() { + return autoCommit; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteRequest.java index 1fde9084299a0..7034b86cd848e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcQueryExecuteRequest.java @@ -37,6 +37,10 @@ public class OdbcQueryExecuteRequest extends OdbcRequest { @GridToStringExclude private final Object[] args; + /** Autocommit flag. */ + @GridToStringInclude + private final boolean autoCommit; + /** Query timeout in seconds. */ @GridToStringInclude private final int timeout; @@ -47,7 +51,8 @@ public class OdbcQueryExecuteRequest extends OdbcRequest { * @param args Arguments list. * @param timeout Timeout in seconds. */ - public OdbcQueryExecuteRequest(@Nullable String schema, String sqlQry, Object[] args, int timeout) { + public OdbcQueryExecuteRequest(@Nullable String schema, String sqlQry, Object[] args, int timeout, + boolean autoCommit) { super(QRY_EXEC); assert sqlQry != null : "SQL query should not be null"; @@ -56,6 +61,7 @@ public OdbcQueryExecuteRequest(@Nullable String schema, String sqlQry, Object[] this.sqlQry = sqlQry; this.args = args; this.timeout = timeout; + this.autoCommit = autoCommit; } /** @@ -91,4 +97,10 @@ public int timeout() { return S.toString(OdbcQueryExecuteRequest.class, this, "args", args, true); } + /** + * @return Autocommit flag. + */ + public boolean autoCommit() { + return autoCommit; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java index b0f27d08601ae..9510e633ad0c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java @@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.FieldsQueryCursor; @@ -46,7 +47,9 @@ import org.apache.ignite.internal.processors.query.GridQueryProperty; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.processors.query.NestedTxMode; import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; @@ -76,6 +79,9 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler { /** Busy lock. */ private final GridSpinBusyLock busyLock; + /** Worker. */ + private final OdbcRequestHandlerWorker worker; + /** Maximum allowed cursors. */ private final int maxCursors; @@ -91,6 +97,9 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler { /** Replicated only flag. */ private final boolean replicatedOnly; + /** Nested transaction behaviour. */ + private final NestedTxMode nestedTxMode; + /** Collocated flag. */ private final boolean collocated; @@ -101,7 +110,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler { private final boolean skipReducerOnUpdate; /** Authentication context */ - private AuthorizationContext actx; + private final AuthorizationContext actx; /** Client version. */ private ClientListenerProtocolVersion ver; @@ -117,12 +126,13 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler { * @param collocated Collocated flag. * @param lazy Lazy flag. * @param skipReducerOnUpdate Skip reducer on update flag. + * @param nestedTxMode Nested transaction mode. * @param actx Authentication context. * @param ver Client protocol version. */ public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors, boolean distributedJoins, boolean enforceJoinOrder, boolean replicatedOnly, boolean collocated, boolean lazy, - boolean skipReducerOnUpdate, AuthorizationContext actx, ClientListenerProtocolVersion ver) { + boolean skipReducerOnUpdate, AuthorizationContext actx, NestedTxMode nestedTxMode, ClientListenerProtocolVersion ver) { this.ctx = ctx; this.busyLock = busyLock; this.maxCursors = maxCursors; @@ -133,20 +143,56 @@ public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int this.lazy = lazy; this.skipReducerOnUpdate = skipReducerOnUpdate; this.actx = actx; + this.nestedTxMode = nestedTxMode; this.ver = ver; log = ctx.log(getClass()); + + if (ctx.grid().configuration().isMvccEnabled()) + worker = new OdbcRequestHandlerWorker(ctx.igniteInstanceName(), log, this, ctx); + else + worker = null; } /** {@inheritDoc} */ @Override public ClientListenerResponse handle(ClientListenerRequest req0) { assert req0 != null; + assert req0 instanceof OdbcRequest; + OdbcRequest req = (OdbcRequest)req0; + if (worker == null) + return doHandle(req); + else { + GridFutureAdapter fut = worker.process(req); + + try { + return fut.get(); + } + catch (IgniteCheckedException e) { + return exceptionToResult(e); + } + } + } + + /** + * Start worker, if it's present. + */ + void start() { + if (worker != null) + worker.start(); + } + + /** + * Handle ODBC request. + * @param req ODBC request. + * @return Response. + */ + public ClientListenerResponse doHandle(OdbcRequest req) { if (!busyLock.enterBusy()) return new OdbcResponse(IgniteQueryErrorCode.UNKNOWN, - "Failed to handle ODBC request because node is stopping: " + req); + "Failed to handle ODBC request because node is stopping: " + req); if (actx != null) AuthorizationContext.context(actx); @@ -204,6 +250,17 @@ public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int public void onDisconnect() { if (busyLock.enterBusy()) { + if (worker != null) { + worker.cancel(); + + try { + worker.join(); + } + catch (InterruptedException e) { + // No-op. + } + } + try { for (OdbcQueryResults res : qryResults.values()) @@ -220,9 +277,11 @@ public void onDisconnect() { * @param schema Schema. * @param sql SQL request. * @param args Arguments. + * @param autoCommit Autocommit transaction. + * @param timeout Query timeout. * @return Query instance. */ - private SqlFieldsQueryEx makeQuery(String schema, String sql, Object[] args, int timeout) { + private SqlFieldsQueryEx makeQuery(String schema, String sql, Object[] args, int timeout, boolean autoCommit) { SqlFieldsQueryEx qry = new SqlFieldsQueryEx(sql, null); qry.setArgs(args); @@ -234,6 +293,8 @@ private SqlFieldsQueryEx makeQuery(String schema, String sql, Object[] args, int qry.setLazy(lazy); qry.setSchema(schema); qry.setSkipReducerOnUpdate(skipReducerOnUpdate); + qry.setNestedTxMode(nestedTxMode); + qry.setAutoCommit(autoCommit); qry.setTimeout(timeout, TimeUnit.SECONDS); @@ -264,7 +325,7 @@ private ClientListenerResponse executeQuery(OdbcQueryExecuteRequest req) { log.debug("ODBC query parsed [reqId=" + req.requestId() + ", original=" + req.sqlQuery() + ", parsed=" + sql + ']'); - SqlFieldsQuery qry = makeQuery(req.schema(), sql, req.arguments(), req.timeout()); + SqlFieldsQuery qry = makeQuery(req.schema(), sql, req.arguments(), req.timeout(), req.autoCommit()); List>> cursors = ctx.query().querySqlFields(qry, true, false); @@ -313,7 +374,7 @@ private ClientListenerResponse executeBatchQuery(OdbcQueryExecuteBatchRequest re log.debug("ODBC query parsed [reqId=" + req.requestId() + ", original=" + req.sqlQuery() + ", parsed=" + sql + ']'); - SqlFieldsQueryEx qry = makeQuery(req.schema(), sql, null, req.timeout()); + SqlFieldsQueryEx qry = makeQuery(req.schema(), sql, null, req.timeout(), req.autoCommit()); Object[][] paramSet = req.arguments(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandlerWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandlerWorker.java new file mode 100644 index 0000000000000..4184b6a9ad61b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandlerWorker.java @@ -0,0 +1,131 @@ +/* + * 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.odbc; + +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.odbc.ClientListenerNioListener; +import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.thread.IgniteThread; +import org.jetbrains.annotations.Nullable; + +import java.util.concurrent.LinkedBlockingQueue; + +/** + * ODBC request handler worker to maintain single threaded transactional execution of SQL statements when MVCC is on.

    + * This worker is intended for internal use as a temporary solution and from within {@link OdbcRequestHandler}, + * therefore it does not do any fine-grained lifecycle handling as it relies on existing guarantees from + * {@link ClientListenerNioListener}. + */ +class OdbcRequestHandlerWorker extends GridWorker { + /** Requests queue.*/ + private final LinkedBlockingQueue>> queue = + new LinkedBlockingQueue<>(); + + /** Handler.*/ + private final OdbcRequestHandler hnd; + + /** Context.*/ + private final GridKernalContext ctx; + + /** Response */ + private final static ClientListenerResponse ERR_RESPONSE = new OdbcResponse(IgniteQueryErrorCode.UNKNOWN, + "Connection closed."); + + /** + * Constructor. + * @param igniteInstanceName Instance name. + * @param log Logger. + * @param hnd Handler. + * @param ctx Kernal context. + */ + OdbcRequestHandlerWorker(@Nullable String igniteInstanceName, IgniteLogger log, OdbcRequestHandler hnd, + GridKernalContext ctx) { + super(igniteInstanceName, "odbc-request-handler-worker", log); + + A.notNull(hnd, "hnd"); + + this.hnd = hnd; + + this.ctx = ctx; + } + + /** + * Start this worker. + */ + void start() { + new IgniteThread(this).start(); + } + + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + try { + while (!isCancelled()) { + T2> req = queue.take(); + + GridFutureAdapter fut = req.get2(); + + try { + ClientListenerResponse res = hnd.doHandle(req.get1()); + + fut.onDone(res); + } + catch (Exception e) { + fut.onDone(e); + } + } + } + finally { + // Notify indexing that this worker is being stopped. + try { + ctx.query().getIndexing().onClientDisconnect(); + } + catch (Exception e) { + // No-op. + } + + // Drain the queue on stop. + T2> req = queue.poll(); + + while (req != null) { + req.get2().onDone(ERR_RESPONSE); + + req = queue.poll(); + } + } + } + + /** + * Initiate request processing. + * @param req Request. + * @return Future to track request processing. + */ + GridFutureAdapter process(OdbcRequest req) { + GridFutureAdapter fut = new GridFutureAdapter<>(); + + queue.add(new T2<>(req, fut)); + + return fut; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/EnlistOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/EnlistOperation.java new file mode 100644 index 0000000000000..fdb6f1e580772 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/EnlistOperation.java @@ -0,0 +1,91 @@ +/* + * 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; + +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.jetbrains.annotations.Nullable; + +/** + * Operations on entries which could be performed during transaction. + * Operations are used during SQL statements execution, but does not define exact SQL statements semantics. + * It is better to treat them independently and having their own semantics. + */ +public enum EnlistOperation { + /** + * This operation creates entry if it does not exist or raises visible failure otherwise. + */ + INSERT(GridCacheOperation.CREATE), + /** + * This operation creates entry if it does not exist or modifies existing one otherwise. + */ + UPSERT(GridCacheOperation.UPDATE), + /** + * This operation modifies existing entry or does nothing if entry does not exist. + */ + UPDATE(GridCacheOperation.UPDATE), + /** + * This operation deletes existing entry or does nothing if entry does not exist. + */ + DELETE(GridCacheOperation.DELETE), + /** + * This operation locks existing entry protecting it from updates by other transactions + * or does notrhing if entry does not exist. + */ + LOCK(null); + + /** */ + private final GridCacheOperation cacheOp; + + /** */ + EnlistOperation(GridCacheOperation cacheOp) { + this.cacheOp = cacheOp; + } + + /** + * @return Corresponding Cache operation. + */ + public GridCacheOperation cacheOperation() { + return cacheOp; + } + + /** */ + public boolean isDeleteOrLock() { + return this == DELETE || this == LOCK; + } + + /** + * Indicates that an operation cannot create new row. + */ + public boolean noCreate() { + // has no meaning for LOCK + assert this != LOCK; + + return this == UPDATE || this == DELETE; + } + + /** Enum values. */ + private static final EnlistOperation[] VALS = values(); + + /** + * @param ord Ordinal value. + * @return Enum value. + */ + @Nullable public static EnlistOperation fromOrdinal(int ord) { + return ord < 0 || ord >= VALS.length ? null : VALS[ord]; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java index dedd0752f8684..7aa4021d7b5a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java @@ -31,6 +31,8 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.util.GridSpinBusyLock; @@ -60,6 +62,13 @@ public interface GridQueryIndexing { */ public void stop() throws IgniteCheckedException; + /** + * Performs necessary actions on disconnect of a stateful client (say, one associated with a transaction). + * + * @throws IgniteCheckedException If failed. + */ + public void onClientDisconnect() throws IgniteCheckedException; + /** * Parses SQL query into two step query and executes it. * @@ -79,10 +88,11 @@ public QueryCursor> queryDistributedSql(String schemaNa * @param cliCtx Client context. * @param keepBinary Keep binary flag. * @param failOnMultipleStmts Whether an exception should be thrown for multiple statements query. - * @param cancel Query cancel state handler. @return Cursor. + * @param tracker Query tracker. + * @return Cursor. */ public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel); + SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, MvccQueryTracker tracker, GridQueryCancel cancel); /** * Execute an INSERT statement using data streamer as receiver. @@ -222,6 +232,28 @@ public void registerCache(String cacheName, String schemaName, GridCacheContext< */ public void unregisterCache(GridCacheContext cctx, boolean rmvIdx) throws IgniteCheckedException; + /** + * + * @param cctx Cache context. + * @param ids Involved cache ids. + * @param parts Partitions. + * @param schema Schema name. + * @param qry Query string. + * @param params Query parameters. + * @param flags Flags. + * @param pageSize Fetch page size. + * @param timeout Timeout. + * @param topVer Topology version. + * @param mvccSnapshot MVCC snapshot. + * @param cancel Query cancel object. + * @return Cursor over entries which are going to be changed. + * @throws IgniteCheckedException If failed. + */ + public UpdateSourceIterator prepareDistributedUpdate(GridCacheContext cctx, int[] ids, int[] parts, + String schema, String qry, Object[] params, int flags, + int pageSize, int timeout, AffinityTopologyVersion topVer, + MvccSnapshot mvccSnapshot, GridQueryCancel cancel) throws IgniteCheckedException; + /** * Registers type if it was not known before or updates it otherwise. * @@ -243,7 +275,10 @@ public void registerCache(String cacheName, String schemaName, GridCacheContext< * @param prevRowAvailable Whether previous row is available. * @throws IgniteCheckedException If failed. */ - public void store(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row, CacheDataRow prevRow, + public void store(GridCacheContext cctx, + GridQueryTypeDescriptor type, + CacheDataRow row, + CacheDataRow prevRow, boolean prevRowAvailable) throws IgniteCheckedException; /** 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 57eaa4a5b3079..bd5d91cc653b3 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 @@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; @@ -1752,15 +1753,20 @@ public boolean belongsToTable(GridCacheContext cctx, String expCacheName, String * @param cacheIds Cache IDs. * @return Future that will be completed when rebuilding is finished. */ - public IgniteInternalFuture rebuildIndexesFromHash(Collection cacheIds) { + public IgniteInternalFuture rebuildIndexesFromHash(Set cacheIds) { if (!busyLock.enterBusy()) throw new IllegalStateException("Failed to rebuild indexes from hash (grid is stopping)."); + // Because of alt type ids, there can be few entries in 'types' for a single cache. + // In order to avoid processing a cache more than once, let's track processed names. + Set processedCacheNames = new HashSet<>(); + try { GridCompoundFuture fut = new GridCompoundFuture(); for (Map.Entry e : types.entrySet()) { - if (cacheIds.contains(CU.cacheId(e.getKey().cacheName()))) + if (cacheIds.contains(CU.cacheId(e.getKey().cacheName())) && + processedCacheNames.add(e.getKey().cacheName())) fut.add(rebuildIndexesFromHash(e.getKey().cacheName(), e.getValue())); } @@ -2039,6 +2045,32 @@ private void checkxEnabled() throws IgniteException { INDEXING.module() + " to classpath or moving it from 'optional' to 'libs' folder)."); } + /** + * + * @param cctx Cache context. + * @param cacheIds Involved cache ids. + * @param parts Partitions. + * @param schema Schema name. + * @param qry Query string. + * @param params Query parameters. + * @param flags Flags. + * @param pageSize Fetch page size. + * @param timeout Timeout. + * @param topVer Topology version. + * @param mvccSnapshot MVCC snapshot. + * @param cancel Query cancel object. + * @return Cursor over entries which are going to be changed. + * @throws IgniteCheckedException If failed. + */ + public UpdateSourceIterator prepareDistributedUpdate(GridCacheContext cctx, int[] cacheIds, + int[] parts, String schema, String qry, Object[] params, int flags, int pageSize, int timeout, + AffinityTopologyVersion topVer, MvccSnapshot mvccSnapshot, + GridQueryCancel cancel) throws IgniteCheckedException { + checkxEnabled(); + + return idx.prepareDistributedUpdate(cctx, cacheIds, parts, schema, qry, params, flags, pageSize, timeout, topVer, mvccSnapshot, cancel); + } + /** * Query SQL fields. * @@ -2104,7 +2136,7 @@ public List>> querySqlFields(@Nullable final GridCache GridQueryCancel cancel = new GridQueryCancel(); List>> res = - idx.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, cancel); + idx.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, null, cancel); if (cctx != null) sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx); @@ -2519,15 +2551,15 @@ public PreparedStatement prepareNativeStatement(String schemaName, String sql) t /** * @param cctx Cache context. - * @param val Row. + * @param row Row removed from cache. * @throws IgniteCheckedException Thrown in case of any errors. */ - public void remove(GridCacheContext cctx, CacheDataRow val) + public void remove(GridCacheContext cctx, CacheDataRow row) throws IgniteCheckedException { - assert val != null; + assert row != null; if (log.isDebugEnabled()) - log.debug("Remove [cacheName=" + cctx.name() + ", key=" + val.key()+ ", val=" + val.value() + "]"); + log.debug("Remove [cacheName=" + cctx.name() + ", key=" + row.key()+ ", val=" + row.value() + "]"); if (idx == null) return; @@ -2538,14 +2570,14 @@ public void remove(GridCacheContext cctx, CacheDataRow val) try { QueryTypeDescriptorImpl desc = typeByValue(cctx.name(), cctx.cacheObjectContext(), - val.key(), - val.value(), + row.key(), + row.value(), false); if (desc == null) return; - idx.remove(cctx, desc, val); + idx.remove(cctx, desc, row); } finally { busyLock.leaveBusy(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/NestedTxMode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/NestedTxMode.java new file mode 100644 index 0000000000000..35690030f955a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/NestedTxMode.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query; + +import org.apache.ignite.IgniteException; + +/** + * Behavior options when an attempt to start a nested transaction is made. + */ +public enum NestedTxMode { + /** Previously started transaction will be committed, new transaction will be started. */ + COMMIT, + + /** Warning will be printed to log, no new transaction will be started. */ + IGNORE, + + /** Exception will be thrown, previously started transaction will be rolled back. */ + ERROR; + + /** Default handling mode. */ + public final static NestedTxMode DEFAULT = ERROR; + + /** + * Get enum value from int + * + * @param val Int value. + * @return Enum value. + * @throws IgniteException if the is no enum value associated with the int value. + */ + public static NestedTxMode fromByte(byte val) { + switch (val) { + case 1: + return COMMIT; + + case 2: + return IGNORE; + + case 3: + return ERROR; + + default: + throw new IgniteException("Invalid nested transactions handling mode: " + val); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/UpdateSourceIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/UpdateSourceIterator.java new file mode 100644 index 0000000000000..69feb0fc19435 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/UpdateSourceIterator.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query; + +import java.util.Iterator; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.lang.GridCloseableIterator; +import org.jetbrains.annotations.NotNull; + +/** */ +public interface UpdateSourceIterator extends GridCloseableIterator { + /** + * @return Operation. + */ + public EnlistOperation operation(); + + /** + * Callback method which should be called before moving iteration into another thread. + */ + public default void beforeDetach() { + // No-op. + } + + /** {@inheritDoc} */ + @Override default void close() throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override default boolean isClosed() { + return false; + } + + /** {@inheritDoc} */ + @Override default void removeX() throws IgniteCheckedException { + throw new UnsupportedOperationException("remove"); + } + + /** {@inheritDoc} */ + @Override default boolean hasNext() { + try { + return hasNextX(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override default T next() { + try { + return nextX(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override default void remove() { + try { + removeX(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override @NotNull default Iterator iterator() { + return this; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java index aa668df98b279..1df5c26c5f472 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java @@ -67,12 +67,12 @@ public class GridQueryNextPageResponse implements Message { /** */ private AffinityTopologyVersion retry; - /** Retry cause description*/ - private String retryCause; - /** Last page flag. */ private boolean last; + /** Remove mapping flag. */ + private boolean removeMapping; + /** * For {@link Externalizable}. */ @@ -235,7 +235,7 @@ public Collection plainRows() { writer.incrementState(); case 9: - if (!writer.writeString("retryCause", retryCause)) + if (!writer.writeBoolean("removeMapping", removeMapping)) return false; writer.incrementState(); @@ -323,15 +323,13 @@ public Collection plainRows() { return false; reader.incrementState(); - case 9: - retryCause = reader.readString("retryCause"); + removeMapping = reader.readBoolean("removeMapping"); if (!reader.isLastRead()) return false; reader.incrementState(); - } return reader.afterMessageRead(GridQueryNextPageResponse.class); @@ -362,31 +360,31 @@ public void retry(AffinityTopologyVersion retry) { } /** - * @return Retry Ccause message. + * @return Last page flag. */ - public String retryCause() { - return retryCause; + public boolean last() { + return last; } /** - * @param retryCause Retry Ccause message. + * @param last Last page flag. */ - public void retryCause(String retryCause){ - this.retryCause = retryCause; + public void last(boolean last) { + this.last = last; } /** - * @return Last page flag. + * @param removeMapping Remove mapping flag. */ - public boolean last() { - return last; + public void removeMapping(boolean removeMapping) { + this.removeMapping = removeMapping; } /** - * @param last Last page flag. + * @return Remove mapping flag. */ - public void last(boolean last) { - this.last = last; + public boolean removeMapping() { + return removeMapping; } /** {@inheritDoc} */ @@ -395,4 +393,4 @@ public void last(boolean last) { "valsSize", vals != null ? vals.size() : 0, "rowsSize", plainRows != null ? plainRows.size() : 0); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/subscription/GridInternalSubscriptionProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/subscription/GridInternalSubscriptionProcessor.java index 6c0942a1e7cca..6db7fa5ffcca0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/subscription/GridInternalSubscriptionProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/subscription/GridInternalSubscriptionProcessor.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.cache.persistence.DatabaseLifecycleListener; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetastorageLifecycleListener; import org.jetbrains.annotations.NotNull; @@ -35,6 +36,10 @@ public class GridInternalSubscriptionProcessor extends GridProcessorAdapter { /** */ private List metastorageListeners = new ArrayList<>(); + /** */ + private List databaseListeners = new ArrayList<>(); + + /** * @param ctx Kernal context. */ @@ -54,4 +59,17 @@ public void registerMetastorageListener(@NotNull MetastorageLifecycleListener me public List getMetastorageSubscribers() { return metastorageListeners; } + + /** */ + public void registerDatabaseListener(@NotNull DatabaseLifecycleListener databaseListener) { + if (databaseListener == null) + throw new NullPointerException("Database subscriber should be not-null."); + + databaseListeners.add(databaseListener); + } + + /** */ + public List getDatabaseListeners() { + return databaseListeners; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java index be7648249365e..658e176efa783 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java @@ -18,7 +18,9 @@ package org.apache.ignite.internal.sql; import java.lang.reflect.Field; +import java.util.Collections; import java.util.HashSet; +import java.util.Set; import org.apache.ignite.IgniteException; import org.apache.ignite.internal.util.typedef.F; @@ -38,6 +40,9 @@ public class SqlKeyword { /** Keyword: BATCH_SIZE */ public static final String BATCH_SIZE = "BATCH_SIZE"; + /** Keyword: BEGIN. */ + public static final String BEGIN = "BEGIN"; + /** Keyword: BIGINT */ public static final String BIGINT = "BIGINT"; @@ -65,6 +70,9 @@ public class SqlKeyword { /** Keyword: CHARSET. */ public static final String CHARSET = "CHARSET"; + /** Keyword: COMMIT. */ + public static final String COMMIT = "COMMIT"; + /** Keyword: COPY. */ public static final String COPY = "COPY"; @@ -206,6 +214,9 @@ public class SqlKeyword { /** Keyword: RESTRICT. */ public static final String RESTRICT = "RESTRICT"; + /** Keyword: ROLLBACK. */ + public static final String ROLLBACK = "ROLLBACK"; + /** Keyword: SET. */ public static final String SET = "SET"; @@ -221,6 +232,9 @@ public class SqlKeyword { /** Keyword: SPATIAL. */ public static final String SPATIAL = "SPATIAL"; + /** Keyword: START. */ + public static final String START = "START"; + /** Keyword: STREAMING. */ public static final String STREAMING = "STREAMING"; @@ -236,6 +250,9 @@ public class SqlKeyword { /** Keyword: TINYINT. */ public static final String TINYINT = "TINYINT"; + /** Keyword: TRANSACTION. */ + public static final String TRANSACTION = "TRANSACTION"; + /** Keyword: UNIQUE. */ public static final String UNIQUE = "UNIQUE"; @@ -251,6 +268,9 @@ public class SqlKeyword { /** Keyword: VARCHAR_CASESENSITIVE. */ public static final String VARCHAR_CASESENSITIVE = "VARCHAR_CASESENSITIVE"; + /** Keyword: WORK. */ + public static final String WORK = "WORK"; + /** Keyword: YEAR. */ public static final String YEAR = "YEAR"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java index d46863a8e34dc..0be26238bab80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java @@ -17,17 +17,22 @@ package org.apache.ignite.internal.sql; +import org.apache.ignite.internal.sql.command.SqlBeginTransactionCommand; import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; import org.apache.ignite.internal.sql.command.SqlAlterUserCommand; import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.sql.command.SqlCommand; +import org.apache.ignite.internal.sql.command.SqlCommitTransactionCommand; import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand; import org.apache.ignite.internal.sql.command.SqlCreateUserCommand; import org.apache.ignite.internal.sql.command.SqlDropIndexCommand; import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand; +import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand; import org.apache.ignite.internal.sql.command.SqlDropUserCommand; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.sql.SqlKeyword.BEGIN; +import static org.apache.ignite.internal.sql.SqlKeyword.COMMIT; import static org.apache.ignite.internal.sql.SqlKeyword.ALTER; import static org.apache.ignite.internal.sql.SqlKeyword.COPY; import static org.apache.ignite.internal.sql.SqlKeyword.CREATE; @@ -35,15 +40,21 @@ import static org.apache.ignite.internal.sql.SqlKeyword.HASH; import static org.apache.ignite.internal.sql.SqlKeyword.INDEX; import static org.apache.ignite.internal.sql.SqlKeyword.PRIMARY; +import static org.apache.ignite.internal.sql.SqlKeyword.ROLLBACK; import static org.apache.ignite.internal.sql.SqlKeyword.SET; import static org.apache.ignite.internal.sql.SqlKeyword.SPATIAL; +import static org.apache.ignite.internal.sql.SqlKeyword.START; +import static org.apache.ignite.internal.sql.SqlKeyword.TRANSACTION; import static org.apache.ignite.internal.sql.SqlKeyword.STREAMING; import static org.apache.ignite.internal.sql.SqlKeyword.TABLE; import static org.apache.ignite.internal.sql.SqlKeyword.UNIQUE; +import static org.apache.ignite.internal.sql.SqlKeyword.WORK; import static org.apache.ignite.internal.sql.SqlKeyword.USER; import static org.apache.ignite.internal.sql.SqlParserUtils.errorUnexpectedToken; import static org.apache.ignite.internal.sql.SqlParserUtils.errorUnsupportedIfMatchesKeyword; import static org.apache.ignite.internal.sql.SqlParserUtils.matchesKeyword; +import static org.apache.ignite.internal.sql.SqlParserUtils.skipIfMatchesKeyword; +import static org.apache.ignite.internal.sql.SqlParserUtils.skipIfMatchesOptionalKeyword; /** * SQL parser. @@ -102,6 +113,16 @@ private SqlCommand nextCommand0() { SqlCommand cmd = null; switch (lex.token()) { + case BEGIN: + cmd = processBegin(); + + break; + + case COMMIT: + cmd = processCommit(); + + break; + case CREATE: cmd = processCreate(); @@ -112,6 +133,16 @@ private SqlCommand nextCommand0() { break; + case ROLLBACK: + cmd = processRollback(); + + break; + + case START: + cmd = processStart(); + + break; + case COPY: try { cmd = processCopy(); @@ -139,7 +170,7 @@ private SqlCommand nextCommand0() { return cmd; } else - throw errorUnexpectedToken(lex, CREATE, DROP, ALTER, COPY, SET); + throw errorUnexpectedToken(lex, BEGIN, COMMIT, CREATE, DROP, ROLLBACK, COPY, SET, ALTER, START); case QUOTED: case MINUS: @@ -153,6 +184,30 @@ private SqlCommand nextCommand0() { } } + /** + * Process BEGIN keyword. + * + * @return Command. + */ + private SqlCommand processBegin() { + skipIfMatchesOptionalKeyword(lex, TRANSACTION); + + skipIfMatchesOptionalKeyword(lex, WORK); + + return new SqlBeginTransactionCommand(); + } + + /** + * Process COMMIT keyword. + * + * @return Command. + */ + private SqlCommand processCommit() { + skipIfMatchesOptionalKeyword(lex, TRANSACTION); + + return new SqlCommitTransactionCommand(); + } + /** * Process SET keyword. * @@ -214,7 +269,7 @@ private SqlCommand processCreate() { errorUnsupportedIfMatchesKeyword(lex, HASH, PRIMARY, UNIQUE); } - throw errorUnexpectedToken(lex, INDEX, SPATIAL); + throw errorUnexpectedToken(lex, INDEX, SPATIAL, USER); } /** @@ -242,7 +297,29 @@ private SqlCommand processDrop() { return cmd.parse(lex); } - throw errorUnexpectedToken(lex, INDEX); + throw errorUnexpectedToken(lex, INDEX, USER); + } + + /** + * Process ROLLBACK keyword. + * + * @return Command. + */ + private SqlCommand processRollback() { + skipIfMatchesOptionalKeyword(lex, TRANSACTION); + + return new SqlRollbackTransactionCommand(); + } + + /** + * Process START keyword. + * + * @return Command. + */ + private SqlCommand processStart() { + skipIfMatchesKeyword(lex, TRANSACTION); + + return new SqlBeginTransactionCommand(); } /** @@ -270,6 +347,6 @@ private SqlCommand processAlter() { return cmd.parse(lex); } - throw errorUnexpectedToken(lex, TABLE); + throw errorUnexpectedToken(lex, TABLE, USER); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java index e99af1d8a1988..074bffaafe281 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java @@ -260,6 +260,20 @@ public static void skipIfMatchesKeyword(SqlLexer lex, String expKeyword) { throw errorUnexpectedToken(lex, expKeyword); } + /** + * Skip token if it matches expected keyword by using lookahead. + * If next token is not what we expect, no shift is done. + * + * @param lex Lexer. + * @param expKeyword Expected keyword. + */ + static void skipIfMatchesOptionalKeyword(SqlLexer lex, String expKeyword) { + SqlLexerToken nextTok = lex.lookAhead(); + + if (matchesKeyword(nextTok, expKeyword)) + lex.shift(); + } + /** * Skip next token if it matches expected type. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBeginTransactionCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBeginTransactionCommand.java new file mode 100644 index 0000000000000..e890cc4cbf3b8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBeginTransactionCommand.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.sql.command; + +import org.apache.ignite.internal.sql.SqlLexer; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * BEGIN [TRANSACTION] command. + */ +public class SqlBeginTransactionCommand implements SqlCommand { + /** {@inheritDoc} */ + @Override public SqlCommand parse(SqlLexer lex) { + return this; + } + + /** {@inheritDoc} */ + @Override public String schemaName() { + return null; + } + + /** {@inheritDoc} */ + @Override public void schemaName(String schemaName) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SqlBeginTransactionCommand.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCommitTransactionCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCommitTransactionCommand.java new file mode 100644 index 0000000000000..da14dea036593 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCommitTransactionCommand.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.sql.command; + +import org.apache.ignite.internal.sql.SqlLexer; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * COMMIT command. + */ +public class SqlCommitTransactionCommand implements SqlCommand { + /** {@inheritDoc} */ + @Override public SqlCommand parse(SqlLexer lex) { + return this; + } + + /** {@inheritDoc} */ + @Override public String schemaName() { + return null; + } + + /** {@inheritDoc} */ + @Override public void schemaName(String schemaName) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SqlCommitTransactionCommand.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlRollbackTransactionCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlRollbackTransactionCommand.java new file mode 100644 index 0000000000000..341b794ccd6ef --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlRollbackTransactionCommand.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.sql.command; + +import org.apache.ignite.internal.sql.SqlLexer; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * ROLLBACK command. + */ +public class SqlRollbackTransactionCommand implements SqlCommand { + /** {@inheritDoc} */ + @Override public SqlCommand parse(SqlLexer lex) { + return this; + } + + /** {@inheritDoc} */ + @Override public String schemaName() { + return null; + } + + /** {@inheritDoc} */ + @Override public void schemaName(String schemaName) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SqlRollbackTransactionCommand.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxMvccVersionCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxMvccVersionCheckedException.java new file mode 100644 index 0000000000000..8a0a54b52d309 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxMvccVersionCheckedException.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.transactions; + +import org.apache.ignite.IgniteCheckedException; + +/** + * Exception thrown whenever transaction enters an unknown state. + */ +public class IgniteTxMvccVersionCheckedException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Creates new exception with given error message. + * + * @param msg Error message. + */ + public IgniteTxMvccVersionCheckedException(String msg) { + super(msg); + } + + /** + * Creates new exception with given error message and optional nested exception. + * + * @param msg Error message. + * @param cause Optional nested exception (can be null). + */ + public IgniteTxMvccVersionCheckedException(String msg, Throwable cause) { + super(msg, cause); + } +} \ 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 a7240601ef0c0..87f5882867ab5 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 @@ -285,12 +285,12 @@ private void checkComplete() { onDone(rdc != null ? rdc.reduce() : null); } catch (RuntimeException e) { - logError(null, "Failed to execute compound future reducer: " + this, e); + logError(logger(), "Failed to execute compound future reducer: " + this, e); onDone(e); } catch (AssertionError e) { - logError(null, "Failed to execute compound future reducer: " + this, e); + logError(logger(), "Failed to execute compound future reducer: " + this, e); onDone(e); 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 b36bf16dc9fdb..7b68a6bfd8337 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 @@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; @@ -65,6 +66,9 @@ public class TestRecordingCommunicationSpi extends TcpCommunicationSpi { /** */ private IgniteBiPredicate blockP; + /** */ + private volatile IgniteBiInClosure c; + /** * @param node Node. * @return Test SPI. @@ -87,6 +91,9 @@ public static TestRecordingCommunicationSpi spi(Ignite node) { Message msg0 = ioMsg.message(); + if (c != null) + c.apply(node, msg0); + synchronized (this) { boolean record = (recordClasses != null && recordClasses.contains(msg0.getClass())) || (recordP != null && recordP.apply(node, msg0)); @@ -196,20 +203,16 @@ public void waitForBlocked(Class cls, String nodeName) throws InterruptedExce * @throws InterruptedException If interrupted. */ public void waitForBlocked() throws InterruptedException { - synchronized (this) { - while (blockedMsgs.isEmpty()) - wait(); - } + waitForBlocked(1); } /** - * @param cnt Number of messages to wait. - * + * @param size Number of messages to wait for. * @throws InterruptedException If interrupted. */ - public void waitForBlocked(int cnt) throws InterruptedException { + public void waitForBlocked(int size) throws InterruptedException { synchronized (this) { - while (blockedMsgs.size() < cnt) + while (blockedMsgs.size() < size) wait(); } } @@ -239,6 +242,13 @@ private boolean hasMessage(Class cls, String nodeName) { return false; } + /** + * @param c Message closure. + */ + public void closure(IgniteBiInClosure c) { + this.c = c; + } + /** * @param blockP Message block predicate. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 716d32b588ee7..8bc620fe4b6d2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.UUID; import javax.cache.Cache; import javax.cache.expiry.ExpiryPolicy; @@ -26,7 +27,10 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.eviction.EvictableEntry; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -276,6 +280,12 @@ void recheckLock() { return info; } + /** {@inheritDoc} */ + @Nullable @Override public List allVersionsInfo() + throws IgniteCheckedException { + return null; + } + /** {@inheritDoc} */ @Override public boolean valid(AffinityTopologyVersion topVer) { return true; @@ -391,7 +401,8 @@ void recheckLock() { Object transformClo, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) { + boolean keepBinary, + MvccSnapshot mvccVer) { return val; } @@ -408,6 +419,7 @@ void recheckLock() { String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean keepBinary, + MvccSnapshot mvccVer, @Nullable ReaderArguments args) throws IgniteCheckedException, GridCacheEntryRemovedException { assert false; @@ -425,6 +437,7 @@ void recheckLock() { String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean keepBinary, + MvccSnapshot mvccVer, @Nullable ReaderArguments readerArgs) { assert false; @@ -457,9 +470,41 @@ void recheckLock() { UUID subjId, String taskName, @Nullable GridCacheVersion dhtVer, - @Nullable Long updateCntr) + @Nullable Long updateCntr, + MvccSnapshot mvccVer + ) + throws IgniteCheckedException, GridCacheEntryRemovedException + { + rawPut(val, ttl); + + return new GridCacheUpdateTxResult(true); + } + + /** {@inheritDoc} */ + @Override public GridCacheUpdateTxResult mvccSet(@Nullable IgniteInternalTx tx, UUID affNodeId, CacheObject val, + long ttl0, AffinityTopologyVersion topVer, @Nullable Long updateCntr, MvccSnapshot mvccVer, + GridCacheOperation op, boolean needHistory, + boolean noCreate) throws IgniteCheckedException, GridCacheEntryRemovedException { + rawPut(val, ttl); + + return new GridCacheUpdateTxResult(true); + } + + /** {@inheritDoc} */ + @Override public GridCacheUpdateTxResult mvccRemove(@Nullable IgniteInternalTx tx, UUID affNodeId, + AffinityTopologyVersion topVer, @Nullable Long updateCntr, MvccSnapshot mvccVer, boolean needHistory) throws IgniteCheckedException, GridCacheEntryRemovedException { - return new GridCacheUpdateTxResult(true, rawPut(val, ttl), null); + obsoleteVer = ver; + + val = null; + + return new GridCacheUpdateTxResult(true); + } + + /** {@inheritDoc} */ + @Override public GridCacheUpdateTxResult mvccLock(GridDhtTxLocalAdapter tx, + MvccSnapshot mvccVer) throws GridCacheEntryRemovedException, IgniteCheckedException { + return new GridCacheUpdateTxResult(true); } /** {@inheritDoc} */ @@ -539,15 +584,14 @@ void recheckLock() { UUID subjId, String taskName, @Nullable GridCacheVersion dhtVer, - @Nullable Long updateCntr + @Nullable Long updateCntr, + MvccSnapshot mvccVer ) throws IgniteCheckedException, GridCacheEntryRemovedException { obsoleteVer = ver; - CacheObject old = val; - val = null; - return new GridCacheUpdateTxResult(true, old, null); + return new GridCacheUpdateTxResult(true); } /** @inheritDoc */ @@ -638,6 +682,10 @@ void recheckLock() { @Override public boolean initialValue( CacheObject val, GridCacheVersion ver, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState, long ttl, long expireTime, boolean preload, @@ -891,6 +939,17 @@ GridCacheMvccCandidate anyOwner() { return false; } + /** {@inheritDoc} */ + @Override public GridCacheUpdateTxResult mvccUpdateRowsWithPreloadInfo(IgniteInternalTx tx, + UUID affNodeId, + AffinityTopologyVersion topVer, + @Nullable Long updateCntr, + List entries, + GridCacheOperation op, + MvccSnapshot mvccVer) throws IgniteCheckedException, GridCacheEntryRemovedException { + return null; + } + /** {@inheritDoc} */ @Override public void touch(AffinityTopologyVersion topVer) { context().evicts().touch(this, topVer); 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 4ef1dc3036064..b1df28e37f86b 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 @@ -43,6 +43,8 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryIndexing; @@ -53,6 +55,7 @@ import org.apache.ignite.internal.processors.query.QueryField; import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl; import org.apache.ignite.internal.processors.query.SqlClientContext; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.lang.GridCloseableIterator; @@ -222,6 +225,11 @@ private void checkFailedCache(final Ignite client, final String cacheName) { * To fail on cache start. */ private static class FailedIndexing implements GridQueryIndexing { + /** {@inheritDoc} */ + @Override public void onClientDisconnect() throws IgniteCheckedException { + // No-op. + } + /** {@inheritDoc} */ @Override public void start(GridKernalContext ctx, GridSpinBusyLock busyLock) throws IgniteCheckedException { // No-op @@ -240,7 +248,7 @@ private static class FailedIndexing implements GridQueryIndexing { /** {@inheritDoc} */ @Override public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { + SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, MvccQueryTracker tracker, GridQueryCancel cancel) { return null; } @@ -311,6 +319,14 @@ private static class FailedIndexing implements GridQueryIndexing { // No-op } + /** {@inheritDoc} */ + @Override public UpdateSourceIterator prepareDistributedUpdate(GridCacheContext cctx, int[] ids, int[] parts, + String schema, String qry, Object[] params, int flags, int pageSize, int timeout, + AffinityTopologyVersion topVer, + MvccSnapshot mvccVer, GridQueryCancel cancel) throws IgniteCheckedException { + return null; + } + /** {@inheritDoc} */ @Override public boolean registerType(GridCacheContext cctx, GridQueryTypeDescriptor desc) throws IgniteCheckedException { @@ -319,7 +335,7 @@ private static class FailedIndexing implements GridQueryIndexing { /** {@inheritDoc} */ @Override public void store(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row, - CacheDataRow prevRow, boolean prevRowAvailable) { + CacheDataRow prevRow, boolean prevRowAvailable) throws IgniteCheckedException { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java new file mode 100644 index 0000000000000..e51b7d04203f9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java @@ -0,0 +1,685 @@ +/* + * 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.mvcc; + +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.ThreadLocalRandom; +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.cluster.ClusterNode; +import org.apache.ignite.cluster.ClusterTopologyException; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.distributed.TestCacheNodeExcludingFilter; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccSnapshotResponse; +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.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; +import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE; + +/** + * Base class for Mvcc coordinator failover test. + */ +public abstract class CacheMvccAbstractBasicCoordinatorFailoverTest extends CacheMvccAbstractTest { + /** + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + protected void coordinatorFailureSimple( + final TransactionConcurrency concurrency, + final TransactionIsolation isolation, + ReadMode readMode, + WriteMode writeMode + ) throws Exception { + testSpi = true; + + startGrids(3); + + client = true; + + final Ignite client = startGrid(3); + + final IgniteCache cache = client.createCache( + cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class)); + + final Integer key1 = primaryKey(jcache(1)); + final Integer key2 = primaryKey(jcache(2)); + + TestRecordingCommunicationSpi crdSpi = TestRecordingCommunicationSpi.spi(ignite(0)); + + crdSpi.blockMessages(MvccSnapshotResponse.class, client.name()); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + try { + try (Transaction tx = client.transactions().txStart(concurrency, isolation)) { + writeByMode(cache, key1, 1, writeMode, INTEGER_CODEC); + writeByMode(cache, key2, 2, writeMode, INTEGER_CODEC); + + tx.commit(); + } + + fail(); + } + catch (ClusterTopologyException e) { + info("Expected exception: " + e); + + assertNotNull(e.retryReadyFuture()); + + e.retryReadyFuture().get(); + } + catch (CacheException e) { + info("Expected exception: " + e); + } + catch (Throwable e) { + fail("Unexpected exception: " + e); + } + + return null; + } + }, "tx-thread"); + + crdSpi.waitForBlocked(); + + stopGrid(0); + + fut.get(); + + assertNull(readByMode(cache, key1, readMode, INTEGER_CODEC)); + assertNull(readByMode(cache, key2, readMode, INTEGER_CODEC)); + + try (Transaction tx = client.transactions().txStart(concurrency, isolation)) { + writeByMode(cache, key1, 1, writeMode, INTEGER_CODEC); + writeByMode(cache, key2, 2, writeMode, INTEGER_CODEC); + + tx.commit(); + } + + assertEquals(1, readByMode(cache, key1, readMode, INTEGER_CODEC)); + assertEquals(2, readByMode(cache, key2, readMode, INTEGER_CODEC)); + } + + /** + * @param readDelay {@code True} if delays get requests. + * @param readInTx {@code True} to read inside transaction. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + protected void readInProgressCoordinatorFails(boolean readDelay, + final boolean readInTx, + final TransactionConcurrency concurrency, + final TransactionIsolation isolation, + ReadMode readMode, + WriteMode writeMode, + @Nullable IgniteInClosure cfgC) throws Exception { + final int COORD_NODES = 5; + final int SRV_NODES = 4; + + if (readDelay) + testSpi = true; + + startGrids(COORD_NODES); + + startGridsMultiThreaded(COORD_NODES, SRV_NODES); + + client = true; + + Ignite client = startGrid(COORD_NODES + SRV_NODES); + + final List cacheNames = new ArrayList<>(); + + final int KEYS = 100; + + final Map vals = new HashMap<>(); + + for (int i = 0; i < KEYS; i++) + vals.put(i, 0); + + String[] exclude = new String[COORD_NODES]; + + for (int i = 0; i < COORD_NODES; i++) + exclude[i] = testNodeName(i); + + for (CacheConfiguration ccfg : cacheConfigurations()) { + ccfg.setName("cache-" + cacheNames.size()); + + if (cfgC != null) + cfgC.apply(ccfg); + + // First server nodes are 'dedicated' coordinators. + ccfg.setNodeFilter(new TestCacheNodeExcludingFilter(exclude)); + + cacheNames.add(ccfg.getName()); + + IgniteCache cache = client.createCache(ccfg); + + boolean updated = false; + + while (!updated) { + try (Transaction tx = client.transactions().txStart(concurrency, isolation)) { + tx.timeout(TX_TIMEOUT); + + writeAllByMode(cache, vals, writeMode, INTEGER_CODEC); + + tx.commit(); + + updated = true; + } + catch (Exception e) { + handleTxException(e); + } + } + } + + if (readDelay) { + for (int i = COORD_NODES; i < COORD_NODES + SRV_NODES + 1; i++) { + TestRecordingCommunicationSpi.spi(ignite(i)).closure(new IgniteBiInClosure() { + @Override public void apply(ClusterNode node, Message msg) { + if (msg instanceof GridNearGetRequest) + doSleep(ThreadLocalRandom.current().nextLong(50) + 1); + } + }); + } + } + + final AtomicBoolean done = new AtomicBoolean(); + + try { + final AtomicInteger readNodeIdx = new AtomicInteger(0); + + IgniteInternalFuture getFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + try { + Ignite node = ignite(COORD_NODES + (readNodeIdx.getAndIncrement() % (SRV_NODES + 1))); + + int cnt = 0; + + while (!done.get() && !Thread.currentThread().isInterrupted()) { + for (String cacheName : cacheNames) { + IgniteCache cache = node.cache(cacheName); + + Map res = null; + + if (readInTx) { + try (Transaction tx = node.transactions().txStart(concurrency, isolation)) { + tx.timeout(TX_TIMEOUT); + + res = readAllByMode(cache, vals.keySet(), readMode, INTEGER_CODEC); + + tx.commit(); + } + catch (Exception e) { // TODO Remove catch clause when IGNITE-8841 implemented. + handleTxException(e); + } + } + else + res = readAllByMode(cache, vals.keySet(), readMode, INTEGER_CODEC); + + if (readInTx) { // TODO IGNITE-8841 + assertTrue("res.size=" + (res == null ? 0 : res.size()) + ", res=" + res, res == null || vals.size() == res.size()); + } + else { + assertEquals(vals.size(), res.size()); + + Integer val0 = null; + + for (Integer val : res.values()) { + if (val0 == null) + val0 = val; + else + assertEquals(val0, val); + } + } + } + + cnt++; + } + + log.info("Finished [node=" + node.name() + ", readCnt=" + cnt + ']'); + + return null; + } + catch (Throwable e) { + error("Unexpected error: " + e, e); + + throw e; + } + } + }, ((SRV_NODES + 1) + 1) * 2, "get-thread"); + + IgniteInternalFuture putFut1 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + Ignite node = ignite(COORD_NODES); + + List caches = new ArrayList<>(); + + for (String cacheName : cacheNames) + caches.add(node.cache(cacheName)); + + Integer val = 1; + + while (!done.get()) { + Map vals = new HashMap<>(); + + for (int i = 0; i < KEYS; i++) + vals.put(i, val); + + for (IgniteCache cache : caches) { + try { + try (Transaction tx = node.transactions().txStart(concurrency, isolation)) { + tx.timeout(TX_TIMEOUT); + + writeAllByMode(cache, vals, writeMode, INTEGER_CODEC); + + tx.commit(); + } + } + catch (Exception e) { + handleTxException(e); + } + } + + val++; + } + + return null; + } + }, "putAll-thread"); + + IgniteInternalFuture putFut2 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + Ignite node = ignite(COORD_NODES); + + IgniteCache cache = node.cache(cacheNames.get(0)); + + Integer val = 0; + + while (!done.get()) { + try { + try (Transaction tx = node.transactions().txStart(concurrency, isolation)) { + tx.timeout(TX_TIMEOUT); + + writeByMode(cache, Integer.MAX_VALUE, val, writeMode, INTEGER_CODEC); + + tx.commit(); + } + } + catch (Exception e) { + handleTxException(e); + } + + val++; + } + + return null; + } + }, "put-thread"); + + for (int i = 0; i < COORD_NODES && !getFut.isDone(); i++) { + U.sleep(3000); + + stopGrid(i); + + awaitPartitionMapExchange(); + } + + done.set(true); + + getFut.get(); + putFut1.get(); + putFut2.get(); + + for (Ignite node : G.allGrids()) + checkActiveQueriesCleanup(node); + } + finally { + done.set(true); + } + } + + /** + * @param concurrency Tx concurrency level. + * @param isolation Tx isolation level. + * @param cfgC Cache cfg closure. + * @param readMode Read mode. + * @param writeMode Write mode. + * @throws Exception If failed. + */ + protected void txInProgressCoordinatorChangeSimple( + final TransactionConcurrency concurrency, + final TransactionIsolation isolation, + @Nullable IgniteInClosure cfgC, + ReadMode readMode, + WriteMode writeMode) throws Exception { + MvccProcessorImpl.coordinatorAssignClosure(new CoordinatorAssignClosure()); + + Ignite srv0 = startGrids(4); + + client = true; + + startGrid(4); + + client = false; + + nodeAttr = CRD_ATTR; + + int crdIdx = 5; + + startGrid(crdIdx); + + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setNodeFilter(new CoordinatorNodeFilter()); + + if (cfgC != null) + cfgC.apply(ccfg); + + srv0.createCache(ccfg); + + Set keys = F.asSet(1, 2, 3); + + for (int i = 0; i < 5; i++) { + Ignite node = ignite(i); + + info("Test with node: " + node.name()); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = node.transactions().txStart(concurrency, isolation)) { + assertTrue(readAllByMode(cache, keys, readMode, INTEGER_CODEC).isEmpty()); + + startGrid(crdIdx + 1); + + stopGrid(crdIdx); + + crdIdx++; + + tx.commit(); + } + catch (Exception e) { + handleTxException(e); + } + + checkActiveQueriesCleanup(ignite(crdIdx)); + } + } + + /** + * @param fromClient {@code True} if read from client node, otherwise from server node. + * @throws Exception If failed. + */ + protected void readInProgressCoordinatorFailsSimple(boolean fromClient, + @Nullable IgniteInClosure cfgC, + ReadMode readMode, + WriteMode writeMode) throws Exception { + for (boolean readInTx : new boolean[]{false, true}) { + for (int i = 1; i <= 3; i++) { + readInProgressCoordinatorFailsSimple(fromClient, i, readInTx,cfgC, readMode, writeMode); + + afterTest(); + } + } + } + + /** + * @param fromClient {@code True} if read from client node, otherwise from server node. + * @param crdChangeCnt Number of coordinator changes. + * @param readInTx {@code True} to read inside transaction. + * @param cfgC Cache configuration closure. + * @param readMode Read mode. + * @param writeMode Write mode. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void readInProgressCoordinatorFailsSimple(boolean fromClient, + int crdChangeCnt, + final boolean readInTx, + @Nullable IgniteInClosure cfgC, + ReadMode readMode, + WriteMode writeMode) throws Exception { + info("readInProgressCoordinatorFailsSimple [fromClient=" + fromClient + + ", crdChangeCnt=" + crdChangeCnt + + ", readInTx=" + readInTx + ']'); + + TransactionConcurrency concurrency = readMode == ReadMode.GET ? OPTIMISTIC : PESSIMISTIC; // TODO IGNITE-7184 + TransactionIsolation isolation = readMode == ReadMode.GET ? SERIALIZABLE : REPEATABLE_READ; // TODO IGNITE-7184 + + testSpi = true; + + client = false; + + final int SRVS = 3; + final int COORDS = crdChangeCnt + 1; + + startGrids(SRVS + COORDS); + + client = true; + + assertTrue(startGrid(SRVS + COORDS).configuration().isClientMode()); + + final Ignite getNode = fromClient ? ignite(SRVS + COORDS) : ignite(COORDS); + + String[] excludeNodes = new String[COORDS]; + + for (int i = 0; i < COORDS; i++) + excludeNodes[i] = testNodeName(i); + + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setNodeFilter(new TestCacheNodeExcludingFilter(excludeNodes)); + + if (cfgC != null) + cfgC.apply(ccfg); + + final IgniteCache cache = getNode.createCache(ccfg); + + final Set keys = new HashSet<>(); + + List keys1 = primaryKeys(jcache(COORDS), 10); + List keys2 = primaryKeys(jcache(COORDS + 1), 10); + + keys.addAll(keys1); + keys.addAll(keys2); + + Map vals = new HashMap(); + + for (Integer key : keys) + vals.put(key, -1); + + try (Transaction tx = getNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + writeAllByMode(cache, vals, writeMode, INTEGER_CODEC); + + tx.commit(); + } + + final TestRecordingCommunicationSpi getNodeSpi = TestRecordingCommunicationSpi.spi(getNode); + + getNodeSpi.blockMessages(new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode node, Message msg) { + String msgClsName = msg.getClass().getSimpleName(); + + return msgClsName.matches("GridNearGetRequest|GridH2QueryRequest|GridCacheQueryRequest"); + } + }); + + IgniteInternalFuture getFut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + Map res = null; + + if (readInTx) { + try (Transaction tx = getNode.transactions().txStart(concurrency, isolation)) { + res = readAllByMode(cache, keys, readMode, INTEGER_CODEC); + + tx.rollback(); + } + catch (Exception e) { + handleTxException(e); + } + } + else + res = readAllByMode(cache, keys, readMode, INTEGER_CODEC); + + assertTrue((res != null || readInTx) || (res != null && 20 == res.size())); + + if (res != null) { + Integer val = null; + + for (Integer val0 : res.values()) { + assertNotNull(val0); + + if (val == null) + val = val0; + else + assertEquals("res=" + res, val, val0); + } + } + + return null; + } + }, "get-thread"); + + getNodeSpi.waitForBlocked(); + + for (int i = 0; i < crdChangeCnt; i++) + stopGrid(i); + + for (int i = 0; i < 10; i++) { + vals = new HashMap(); + + for (Integer key : keys) + vals.put(key, i); + + while (true) { + try (Transaction tx = getNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + writeAllByMode(cache, vals, writeMode, INTEGER_CODEC); + + tx.commit(); + + break; + } + catch (Exception e) { + handleTxException(e); + } + } + } + + getNodeSpi.stopBlock(true); + + getFut.get(); + + for (Ignite node : G.allGrids()) + checkActiveQueriesCleanup(node); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + protected void checkCoordinatorChangeActiveQueryClientFails_Simple(@Nullable IgniteInClosure cfgC, + ReadMode readMode, + WriteMode writeMode) throws Exception { + testSpi = true; + + client = false; + + final int SRVS = 3; + final int COORDS = 1; + + startGrids(SRVS + COORDS); + + client = true; + + Ignite client = startGrid(SRVS + COORDS); + + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setNodeFilter(new TestCacheNodeExcludingFilter(testNodeName(0))); + + if (cfgC != null) + cfgC.apply(ccfg); + + final IgniteCache cache = client.createCache(ccfg); + + final Map vals = new HashMap(); + + for (int i = 0; i < 100; i++) + vals.put(i, i); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + writeAllByMode(cache, vals, writeMode, INTEGER_CODEC); + + tx.commit(); + } + + final TestRecordingCommunicationSpi clientSpi = TestRecordingCommunicationSpi.spi(client); + + clientSpi.blockMessages(new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode node, Message msg) { + String msgClsName = msg.getClass().getSimpleName(); + + return msgClsName.matches("GridNearGetRequest|GridH2QueryRequest|GridCacheQueryRequest"); + } + }); + + IgniteInternalFuture getFut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + Map res = readAllByMode(cache, vals.keySet(), readMode, INTEGER_CODEC); + + assertEquals(vals, res); + + return null; + } + }, "get-thread"); + + clientSpi.waitForBlocked(); + + stopGrid(0); + + clientSpi.stopBlock(true); + + getFut.get(); + + for (Ignite node : G.allGrids()) + checkActiveQueriesCleanup(node); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractCoordinatorFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractCoordinatorFailoverTest.java new file mode 100644 index 0000000000000..54e43150781c0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractCoordinatorFailoverTest.java @@ -0,0 +1,148 @@ +/* + * 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.mvcc; + +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.GET; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SCAN; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.PUT; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; +import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE; + +/** + * Mvcc cache API coordinator failover test. + */ +public abstract class CacheMvccAbstractCoordinatorFailoverTest extends CacheMvccAbstractBasicCoordinatorFailoverTest { + /** + * @throws Exception If failed. + */ + public void testAccountsTxGet_Server_Backups0_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + accountsTxReadAll(2, 0, 0, 64, + null, true, GET, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGet_SingleNode_CoordinatorFails() throws Exception { + accountsTxReadAll(1, 0, 0, 1, + null, true, GET, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_Server_Backups0_CoordinatorFails() throws Exception { + accountsTxReadAll(2, 0, 0, 64, + null, true, SCAN, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_SingleNode_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + accountsTxReadAll(1, 0, 0, 1, + null, true, SCAN, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_Server_Backups0_RestartCoordinator_GetPut() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD , 2, 0, 0, 64, + null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_SingleNode_RestartCoordinator_GetPut_Persistence() throws Exception { + persistence = true; + + putAllGetAll(RestartMode.RESTART_CRD , 1, 0, 0, 1, + null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_Servers_Backups0__PutGet_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + updateNObjectsTest(5, 2, 0, 0, 64, DFLT_TEST_TIME, + null, GET, PUT, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_SingleNode__PutGet_CoordinatorFails() throws Exception { + + updateNObjectsTest(7, 1, 0, 0, 1, DFLT_TEST_TIME, + null, GET, PUT, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testCoordinatorFailureSimplePessimisticTxPutGet() throws Exception { + coordinatorFailureSimple(PESSIMISTIC, REPEATABLE_READ, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testCoordinatorFailureSimpleSerializableTxPutGet() throws Exception { + coordinatorFailureSimple(OPTIMISTIC, SERIALIZABLE, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testCoordinatorFailureSimpleOptimisticTxPutGet() throws Exception { + coordinatorFailureSimple(OPTIMISTIC, REPEATABLE_READ, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testTxInProgressCoordinatorChangeSimple_ReadonlyPutGet() throws Exception { + txInProgressCoordinatorChangeSimple(OPTIMISTIC, SERIALIZABLE, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testReadInProgressCoordinatorFailsSimple_FromClientPutGet() throws Exception { + readInProgressCoordinatorFailsSimple(true, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testCoordinatorChangeActiveQueryClientFails_Simple() throws Exception { + checkCoordinatorChangeActiveQueryClientFails_Simple(null, GET, PUT); + } + +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java new file mode 100644 index 0000000000000..f5172c864b4fa --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java @@ -0,0 +1,294 @@ +/* + * 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.mvcc; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteInterruptedException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.lang.IgniteClosure2X; +import org.apache.ignite.internal.util.lang.IgnitePair; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.transactions.Transaction; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * Test that checks + */ +public abstract class CacheMvccAbstractFeatureTest extends CacheMvccAbstractTest { + /** */ + private static final String CACHE_NAME = "Person"; + + /** */ + private Ignite node; + + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrids(4); + + node = grid(0); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); + + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + ccfg.setIndexedTypes(Integer.class, Person.class); + + node.createCache(ccfg); + + for (int i = 0; i < 100; i++) + cache().put(i, new Person("Name" + i, "LastName" + i)); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + node.destroyCache(CACHE_NAME); + + super.afterTest(); + } + + /** + * @param clo Closure to check consistency upon. + * @throws Exception if failed. + */ + void doTestConsistency(IgniteClosure2X clo) throws Exception { + ExecutorService svc = Executors.newFixedThreadPool(2); + + CountDownLatch startLatch = new CountDownLatch(1); + + CountDownLatch endLatch = new CountDownLatch(1); + + try { + Future> fut = svc.submit(new Callable>() { + @Override public IgnitePair call() { + try (Transaction ignored = node.transactions().txStart()) { + // First result that we'll later check w/respect to REPEATABLE READ semantic. + Object res1 = clo.apply(null, null); + + Object res2 = clo.apply(startLatch, endLatch); + + return new IgnitePair<>(res1, res2); + } + } + }); + + svc.submit(new Runnable() { + @Override public void run() { + try { + startLatch.await(); + } + catch (InterruptedException e) { + throw new IgniteInterruptedException(e); + } + + try { + modifyData(jdbcTx()); + } + catch (SQLException e) { + throw new IgniteException(e); + } + + endLatch.countDown(); + } + }).get(); + + IgnitePair res2 = fut.get(); + + assertEquals(res2.get1(), res2.get2()); + } + finally { + svc.shutdown(); + } + } + + /** + * @return Whether native or SQL transactions must be used. + */ + boolean jdbcTx() { + return false; + } + + /** + * @param jdbcTx Whether concurrent transaction must be of SQL type. + */ + private void modifyData(boolean jdbcTx) throws SQLException { + Set keys = new HashSet<>(10); + + for (int i = 0; i < 10; i++) { + int idx; + + do { + idx = (int) (Math.random() * 100) + 1; + } + while (!keys.add(idx)); + } + + if (!jdbcTx) { + try (Transaction ignored = node.transactions().txStart()) { + for (int idx : keys) { + boolean rmv = Math.random() > 0.5; + + if (rmv) + cache().remove(idx); + else { + Person p = cache().get(idx); + + cache().put(idx, new Person(p.fName, p.fName + "Updated")); + } + } + } + } + else { + try (Connection c = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1")) { + c.setAutoCommit(false); + + for (int idx : keys) { + boolean rmv = Math.random() > 0.5; + + if (rmv) { + try (Statement s = c.createStatement()) { + s.execute("DELETE FROM \"Person\".PERSON WHERE _key = " + idx); + } + } + else { + try (Statement s = c.createStatement()) { + s.execute("UPDATE \"Person\".PERSON SET lname = concat(lname, 'Updated')" + + "WHERE _key = " + idx); + } + } + } + + try (Statement s = c.createStatement()) { + s.execute("COMMIT"); + } + } + } + } + + /** + * @return Cache. + */ + IgniteCache cache() { + return node.cache(CACHE_NAME); + } + + /** + * + */ + static class Person implements Serializable { + /** */ + @GridToStringInclude + @QuerySqlField(index = true, groups = "full_name") + private String fName; + + /** */ + @GridToStringInclude + @QuerySqlField(index = true, groups = "full_name") + private String lName; + + /** + * @param fName First name. + * @param lName Last name. + */ + public Person(String fName, String lName) { + this.fName = fName; + this.lName = lName; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(Person.class, this); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + Person person = (Person)o; + + return Objects.equals(fName, person.fName) && + Objects.equals(lName, person.lName); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(fName, lName); + } + } + + /** */ + final static Comparator> ENTRY_CMP = + new Comparator>() { + @Override public int compare(Cache.Entry o1, Cache.Entry o2) { + return o1.getKey().compareTo(o2.getKey()); + } + }; + + /** + * + */ + static List entriesToPersons(List> entries) { + entries.sort(ENTRY_CMP); + + List res = new ArrayList<>(); + + for (Cache.Entry e : entries) + res.add(e.getValue()); + + return res; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java new file mode 100644 index 0000000000000..1abb45f5fd8ed --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java @@ -0,0 +1,2291 @@ +/* + * 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.mvcc; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.Callable; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.FieldsQueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.cluster.ClusterTopologyException; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.lang.GridInClosure3; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; +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.U; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgnitePredicate; +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.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL_SUM; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.DML; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * + */ +public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + protected static final ObjectCodec INTEGER_CODEC = new IntegerCodec(); + + /** */ + protected static final ObjectCodec ACCOUNT_CODEC = new AccountCodec(); + + /** */ + static final int DFLT_PARTITION_COUNT = RendezvousAffinityFunction.DFLT_PARTITION_COUNT; + + /** */ + static final String CRD_ATTR = "testCrd"; + + /** */ + static final long DFLT_TEST_TIME = 30_000; + + /** */ + protected static final int PAGE_SIZE = DataStorageConfiguration.DFLT_PAGE_SIZE; + + /** */ + protected static final int SRVS = 4; + + /** */ + protected boolean client; + + /** */ + protected boolean testSpi; + + /** */ + protected String nodeAttr; + + /** */ + protected boolean persistence; + + /** */ + protected CacheConfiguration ccfg; + + /** */ + protected CacheConfiguration[] ccfgs; + + /** */ + protected boolean disableScheduledVacuum; + + /** */ + protected static final int TX_TIMEOUT = 3000; + + /** + * @return Cache mode. + */ + protected abstract CacheMode cacheMode(); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setMvccEnabled(true); + + if (disableScheduledVacuum) + cfg.setMvccVacuumTimeInterval(Integer.MAX_VALUE); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + if (testSpi) + cfg.setCommunicationSpi(new TestRecordingCommunicationSpi()); + + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + + cfg.setClientMode(client); + + assert (ccfg == null) || (ccfgs == null); + + if (ccfg != null) + cfg.setCacheConfiguration(ccfg); + + if (ccfgs != null) + cfg.setCacheConfiguration(ccfgs); + + if (nodeAttr != null) + cfg.setUserAttributes(F.asMap(nodeAttr, true)); + + DataStorageConfiguration storageCfg = new DataStorageConfiguration(); + + storageCfg.setWalMode(WALMode.LOG_ONLY); + storageCfg.setPageSize(PAGE_SIZE); + + DataRegionConfiguration regionCfg = new DataRegionConfiguration(); + + regionCfg.setPersistenceEnabled(persistence); + regionCfg.setMaxSize(64L * 1024 * 1024); + + storageCfg.setDefaultDataRegionConfiguration(regionCfg); + + cfg.setDataStorageConfiguration(storageCfg); + + cfg.setConsistentId(gridName); + + cfg.setTransactionConfiguration(new TransactionConfiguration() + .setDefaultTxConcurrency(TransactionConcurrency.PESSIMISTIC) + .setDefaultTxIsolation(TransactionIsolation.REPEATABLE_READ)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return DFLT_TEST_TIME + 60_000; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + MvccProcessorImpl.coordinatorAssignClosure(null); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + try { + verifyOldVersionsCleaned(); + + verifyCoordinatorInternalState(); + } + finally { + stopAllGrids(); + } + + MvccProcessorImpl.coordinatorAssignClosure(null); + + cleanPersistenceDir(); + + super.afterTest(); + } + + /** + * @param cfgC Optional closure applied to cache configuration. + * @throws Exception If failed. + */ + final void cacheRecreate(@Nullable IgniteInClosure cfgC) throws Exception { + Ignite srv0 = startGrid(0); + + final int PARTS = 64; + + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, PARTS); + + if (cfgC != null) + cfgC.apply(ccfg); + + IgniteCache cache = (IgniteCache)srv0.createCache(ccfg); + + for (int k = 0; k < PARTS * 2; k++) { + assertNull(cache.get(k)); + + int vals = k % 3 + 1; + + for (int v = 0; v < vals; v++) + cache.put(k, new MvccTestAccount(v, 1)); + + assertEquals(vals - 1, cache.get(k).val); + } + + srv0.destroyCache(cache.getName()); + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, PARTS); + + if (cfgC != null) + cfgC.apply(ccfg); + + cache = (IgniteCache)srv0.createCache(ccfg); + + for (int k = 0; k < PARTS * 2; k++) { + assertNull(cache.get(k)); + + int vals = k % 3 + 2; + + for (int v = 0; v < vals; v++) + cache.put(k, new MvccTestAccount(v + 100, 1)); + + assertEquals(vals - 1 + 100, cache.get(k).val); + } + + srv0.destroyCache(cache.getName()); + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, PARTS); + + IgniteCache cache0 = (IgniteCache)srv0.createCache(ccfg); + + for (long k = 0; k < PARTS * 2; k++) { + assertNull(cache0.get(k)); + + int vals = (int)(k % 3 + 2); + + for (long v = 0; v < vals; v++) + cache0.put(k, v); + + assertEquals((long)(vals - 1), (Object)cache0.get(k)); + } + } + + /** + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param cfgC Optional closure applied to cache configuration. + * @param withRmvs If {@code true} then in addition to puts tests also executes removes. + * @param readMode Read mode. + * @param writeMode Write mode. + * @throws Exception If failed. + */ + final void accountsTxReadAll( + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + @Nullable IgniteInClosure cfgC, + final boolean withRmvs, + final ReadMode readMode, + final WriteMode writeMode + ) throws Exception { + accountsTxReadAll(srvs, clients, cacheBackups, cacheParts, cfgC, withRmvs, readMode, writeMode, DFLT_TEST_TIME, null); + } + + /** + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param cfgC Optional closure applied to cache configuration. + * @param withRmvs If {@code true} then in addition to puts tests also executes removes. + * @param readMode Read mode. + * @param writeMode Write mode. + * @param testTime Test time. + * @throws Exception If failed. + */ + final void accountsTxReadAll( + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + @Nullable IgniteInClosure cfgC, + final boolean withRmvs, + final ReadMode readMode, + final WriteMode writeMode, + long testTime, + RestartMode restartMode + ) throws Exception { + final int ACCOUNTS = 20; + + final int ACCOUNT_START_VAL = 1000; + + final int writers = 4; + + final int readers = 4; + + final IgniteInClosure> init = new IgniteInClosure>() { + @Override public void apply(IgniteCache cache) { + final IgniteTransactions txs = cache.unwrap(Ignite.class).transactions(); + + if (writeMode == WriteMode.PUT) { + Map accounts = new HashMap<>(); + + for (int i = 0; i < ACCOUNTS; i++) + accounts.put(i, new MvccTestAccount(ACCOUNT_START_VAL, 1)); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.putAll(accounts); + + tx.commit(); + } + } + else if (writeMode == WriteMode.DML) { + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + SqlFieldsQuery qry = new SqlFieldsQuery("insert into MvccTestAccount(_key, val, updateCnt) values " + + "(?," + ACCOUNT_START_VAL + ",1)"); + + for (int i = 0; i < ACCOUNTS; i++) { + try (FieldsQueryCursor> cur = cache.query(qry.setArgs(i))) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + } + } + else + assert false : "Unknown write mode"; + } + }; + + final RemovedAccountsTracker rmvdTracker = new RemovedAccountsTracker(ACCOUNTS); + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + cnt++; + + int i1 = rnd.nextInt(ACCOUNTS), i2 = rnd.nextInt(ACCOUNTS); + + while (i2 == i1) + i2 = rnd.nextInt(ACCOUNTS); + + Integer id1 = Math.min(i1, i2); + Integer id2 = Math.max(i1, i2); + + TreeSet keys = new TreeSet<>(); + + keys.add(id1); + keys.add(id2); + + Integer cntr1 = null; + Integer cntr2 = null; + + Integer rmvd = null; + Integer inserted = null; + + MvccTestAccount a1; + MvccTestAccount a2; + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + Map accounts = null; + + if (writeMode == WriteMode.PUT) + accounts = cache.cache.getAll(keys); + else if (writeMode == WriteMode.DML) + accounts = getAllSql(cache); + else + assert false : "Unknown write mode"; + + a1 = accounts.get(id1); + a2 = accounts.get(id2); + + if (!withRmvs) { + assertNotNull(a1); + assertNotNull(a2); + + cntr1 = a1.updateCnt + 1; + cntr2 = a2.updateCnt + 1; + + if (writeMode == WriteMode.PUT) { + cache.cache.put(id1, new MvccTestAccount(a1.val + 1, cntr1)); + cache.cache.put(id2, new MvccTestAccount(a2.val - 1, cntr2)); + } + else if (writeMode == WriteMode.DML) { + updateSql(cache, id1, a1.val + 1, cntr1); + updateSql(cache, id2, a2.val - 1, cntr2); + } + else + assert false : "Unknown write mode"; + } + else { + if (a1 != null || a2 != null) { + if (a1 != null && a2 != null) { + if (rnd.nextInt(10) == 0) { + if (rmvdTracker.size() < ACCOUNTS / 2) { + rmvd = rnd.nextBoolean() ? id1 : id2; + + assertTrue(rmvdTracker.markRemoved(rmvd)); + } + } + + if (rmvd != null) { + if (writeMode == WriteMode.PUT) { + if (rmvd.equals(id1)) { + cache.cache.remove(id1); + cache.cache.put(id2, new MvccTestAccount(a1.val + a2.val, 1)); + } + else { + cache.cache.put(id1, new MvccTestAccount(a1.val + a2.val, 1)); + cache.cache.remove(id2); + } + } + else if (writeMode == WriteMode.DML) { + if (rmvd.equals(id1)) { + removeSql(cache, id1); + updateSql(cache, id2,a1.val + a2.val, 1); + } + else { + updateSql(cache, id1,a1.val + a2.val, 1); + removeSql(cache, id2); + } + } + else + assert false : "Unknown write mode"; + } + else { + if (writeMode == WriteMode.PUT) { + cache.cache.put(id1, new MvccTestAccount(a1.val + 1, 1)); + cache.cache.put(id2, new MvccTestAccount(a2.val - 1, 1)); + } + else if (writeMode == WriteMode.DML) { + updateSql(cache, id1, a1.val + 1, 1); + updateSql(cache, id2, a2.val - 1, 1); + } + else + assert false : "Unknown write mode"; + } + } + else { + if (a1 == null) { + inserted = id1; + + if (writeMode == WriteMode.PUT) { + cache.cache.put(id1, new MvccTestAccount(100, 1)); + cache.cache.put(id2, new MvccTestAccount(a2.val - 100, 1)); + } + else if (writeMode == WriteMode.DML) { + insertSql(cache, id1, 100, 1); + updateSql(cache, id2, a2.val - 100, 1); + } + else + assert false : "Unknown write mode"; + } + else { + inserted = id2; + + if (writeMode == WriteMode.PUT) { + cache.cache.put(id1, new MvccTestAccount(a1.val - 100, 1)); + cache.cache.put(id2, new MvccTestAccount(100, 1)); + } + else if (writeMode == WriteMode.DML) { + updateSql(cache, id1, a1.val - 100, 1); + insertSql(cache, id2, 100, 1); + } + else + assert false : "Unknown write mode"; + } + } + } + } + + tx.commit(); + + // In case of tx success mark inserted. + if (inserted != null) { + assert withRmvs; + + assertTrue(rmvdTracker.unmarkRemoved(inserted)); + } + } + catch (Throwable e) { + if (rmvd != null) { + assert withRmvs; + + // If tx fails, unmark removed. + assertTrue(rmvdTracker.unmarkRemoved(rmvd)); + } + + throw e; + } + + if (!withRmvs) { + Map accounts = null; + + if (writeMode == WriteMode.PUT) + accounts = cache.cache.getAll(keys); + else if (writeMode == WriteMode.DML) + accounts = getAllSql(cache); + else + assert false : "Unknown write mode"; + + a1 = accounts.get(id1); + a2 = accounts.get(id2); + + assertNotNull(a1); + assertNotNull(a2); + + assertTrue(a1.updateCnt >= cntr1); + assertTrue(a2.updateCnt >= cntr2); + } + } + catch (Exception e) { + handleTxException(e); + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Set keys = new LinkedHashSet<>(); + + Map lastUpdateCntrs = new HashMap<>(); + + SqlFieldsQuery sumQry = new SqlFieldsQuery("select sum(val) from MvccTestAccount"); + + while (!stop.get()) { + while (keys.size() < ACCOUNTS) + keys.add(rnd.nextInt(ACCOUNTS)); + + TestCache cache = randomCache(caches, rnd); + + Map accounts = null; + + try { + switch (readMode) { + case GET: { + accounts = cache.cache.getAll(keys); + + break; + } + + case SCAN: { + accounts = new HashMap<>(); + + Iterator> it = cache.cache.iterator(); + + try { + for (; it.hasNext(); ) { + IgniteCache.Entry e = it.next(); + MvccTestAccount old = accounts.put(e.getKey(), e.getValue()); + + assertNull("new=" + e + ", old=" + old, old); + } + } finally { + U.closeQuiet((AutoCloseable) it); + } + + break; + } + + case SQL: { + accounts = new HashMap<>(); + + if (rnd.nextBoolean()) { + SqlQuery qry = + new SqlQuery<>(MvccTestAccount.class, "_key >= 0"); + + for (IgniteCache.Entry e : cache.cache.query(qry).getAll()) { + MvccTestAccount old = accounts.put(e.getKey(), e.getValue()); + + assertNull(old); + } + } + else { + SqlFieldsQuery qry = new SqlFieldsQuery("select _key, val from MvccTestAccount"); + + for (List row : cache.cache.query(qry).getAll()) { + Integer id = (Integer)row.get(0); + Integer val = (Integer)row.get(1); + + MvccTestAccount old = accounts.put(id, new MvccTestAccount(val, 1)); + + assertNull(old); + } + } + + break; + } + + case SQL_SUM: { + BigDecimal sum; + + if (rnd.nextBoolean()) { + List> res = cache.cache.query(sumQry).getAll(); + + assertEquals(1, res.size()); + + sum = (BigDecimal)res.get(0).get(0); + } + else { + Map res = readAllByMode(cache.cache, keys, readMode, ACCOUNT_CODEC); + + sum = (BigDecimal)((Map.Entry)res.entrySet().iterator().next()).getValue(); + } + + assertEquals(ACCOUNT_START_VAL * ACCOUNTS, sum.intValue()); + + break; + } + + default: { + fail(); + + return; + } + } + } + finally { + cache.readUnlock(); + } + + if (accounts != null) { + if (!withRmvs) + assertEquals(ACCOUNTS, accounts.size()); + + int sum = 0; + + for (int i = 0; i < ACCOUNTS; i++) { + MvccTestAccount account = accounts.get(i); + + if (account != null) { + sum += account.val; + + Integer cntr = lastUpdateCntrs.get(i); + + if (cntr != null) + assertTrue(cntr <= account.updateCnt); + + lastUpdateCntrs.put(i, cntr); + } + else + assertTrue(withRmvs); + } + + assertEquals(ACCOUNTS * ACCOUNT_START_VAL, sum); + } + } + + if (idx == 0) { + TestCache cache = randomCache(caches, rnd); + + Map accounts; + + ReadMode readMode0 = readMode == SQL_SUM ? SQL : readMode; + + try { + accounts = readAllByMode(cache.cache, keys, readMode0, ACCOUNT_CODEC);; + } + finally { + cache.readUnlock(); + } + + int sum = 0; + + for (int i = 0; i < ACCOUNTS; i++) { + MvccTestAccount account = accounts.get(i); + + assertTrue(account != null || withRmvs); + + info("Account [id=" + i + ", val=" + (account != null ? account.val : null) + ']'); + + if (account != null) + sum += account.val; + } + + info("Sum: " + sum); + } + } + }; + + readWriteTest( + restartMode, + srvs, + clients, + cacheBackups, + cacheParts, + writers, + readers, + testTime, + cfgC, + init, + writer, + reader); + } + + /** + * Returns all accounts from cache by means of SQL. + * + * @param cache Cache. + * @return All accounts + */ + private static Map getAllSql(TestCache cache) { + Map accounts = new HashMap<>(); + + SqlFieldsQuery qry = new SqlFieldsQuery("select _key, val, updateCnt from MvccTestAccount"); + + for (List row : cache.cache.query(qry).getAll()) { + Integer id = (Integer)row.get(0); + Integer val = (Integer)row.get(1); + Integer updateCnt = (Integer)row.get(2); + + MvccTestAccount old = accounts.put(id, new MvccTestAccount(val, updateCnt)); + + assertNull(old); + } + + return accounts; + } + + /** + * Updates account by means of SQL API. + * + * @param cache Cache. + * @param key Key. + * @param val Value. + * @param updateCnt Update counter. + */ + private static void updateSql(TestCache cache, Integer key, Integer val, Integer updateCnt) { + SqlFieldsQuery qry = new SqlFieldsQuery("update MvccTestAccount set val=" + val + ", updateCnt=" + + updateCnt + " where _key=" + key); + + cache.cache.query(qry).getAll(); + } + + /** + * Removes account by means of SQL API. + * + * @param cache Cache. + * @param key Key. + */ + private static void removeSql(TestCache cache, Integer key) { + SqlFieldsQuery qry = new SqlFieldsQuery("delete from MvccTestAccount where _key=" + key); + + cache.cache.query(qry).getAll(); + } + + /** + * Inserts account by means of SQL API. + * + * @param cache Cache. + * @param key Key. + * @param val Value. + * @param updateCnt Update counter. + */ + private static void insertSql(TestCache cache, int key, Integer val, Integer updateCnt) { + SqlFieldsQuery qry = new SqlFieldsQuery("insert into MvccTestAccount(_key, val, updateCnt) values " + + " (" + key+ ", " + val + ", " + updateCnt + ")"); + + cache.cache.query(qry).getAll(); + } + + /** + * @param restartMode Restart mode. + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param readMode Read mode. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + protected void putAllGetAll( + RestartMode restartMode, + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + @Nullable IgniteInClosure cfgC, + ReadMode readMode, + WriteMode writeMode + ) throws Exception { + final int RANGE = 20; + + final int writers = 4; + + final int readers = 4; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int min = idx * RANGE; + int max = min + RANGE; + + info("Thread range [min=" + min + ", max=" + max + ']'); + + Map map = new HashMap<>(); + + int v = idx * 1_000_000; + + boolean first = true; + + while (!stop.get()) { + while (map.size() < RANGE) + map.put(rnd.nextInt(min, max), v); + + TestCache cache = randomCache(caches, rnd); + + try { + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + if (!first && rnd.nextBoolean()) { + Map res = readAllByMode(cache.cache, map.keySet(), readMode, INTEGER_CODEC); + + for (Integer k : map.keySet()) + assertEquals("res=" + res, v - 1, (Object)res.get(k)); + } + + writeAllByMode(cache.cache, map, writeMode, INTEGER_CODEC); + + tx.commit(); + + first = false; + } + + if (rnd.nextBoolean()) { + Map res = readAllByMode(cache.cache, map.keySet(), readMode, INTEGER_CODEC); + + for (Integer k : map.keySet()) + assertEquals("key=" + k, v, (Object)res.get(k)); + } + + map.clear(); + + v++; + } + catch (Exception e) { + handleTxException(e); + } + finally { + cache.readUnlock(); + + map.clear(); + } + } + + info("Writer done, updates: " + v); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Set keys = new LinkedHashSet<>(); + + Map readVals = new HashMap<>(); + + while (!stop.get()) { + int range = rnd.nextInt(0, writers); + + int min = range * RANGE; + int max = min + RANGE; + + while (keys.size() < RANGE) + keys.add(rnd.nextInt(min, max)); + + TestCache cache = randomCache(caches, rnd); + + Map map; + + try { + map = readAllByMode(cache.cache, keys, readMode, INTEGER_CODEC); + } + catch (Exception e) { + handleTxException(e); + + continue; + } + finally { + cache.readUnlock(); + } + + assertTrue("Invalid map size: " + map.size() + ", map=" + map, map.isEmpty() || map.size() == RANGE); + + Integer val0 = null; + + for (Map.Entry e: map.entrySet()) { + Integer val = e.getValue(); + + assertNotNull(val); + + if (val0 == null) { + Integer readVal = readVals.get(range); + + if (readVal != null) + assertTrue("readVal=" + readVal + ", val=" + val + ", map=" + map,readVal <= val); + + readVals.put(range, val); + + val0 = val; + } + else { + if (!F.eq(val0, val)) { + assertEquals("Unexpected value [range=" + range + ", key=" + e.getKey() + ']' + + ", map=" + map, + val0, + val); + } + } + } + + keys.clear(); + } + } + }; + + readWriteTest( + restartMode, + srvs, + clients, + cacheBackups, + cacheParts, + writers, + readers, + DFLT_TEST_TIME, + cfgC, + null, + writer, + reader); + + for (Ignite node : G.allGrids()) + checkActiveQueriesCleanup(node); + } + + + + /** + * @param N Number of object to update in single transaction. + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param time Test time. + * @param readMode Read mode. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + protected void updateNObjectsTest( + final int N, + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + long time, + @Nullable IgniteInClosure cfgC, + ReadMode readMode, + WriteMode writeMode, + RestartMode restartMode + ) + throws Exception + { + final int TOTAL = 20; + + assert N <= TOTAL; + + info("updateNObjectsTest [n=" + N + ", total=" + TOTAL + ']'); + + final int writers = 4; + + final int readers = 4; + + final IgniteInClosure> init = new IgniteInClosure>() { + @Override public void apply(IgniteCache cache) { + final IgniteTransactions txs = cache.unwrap(Ignite.class).transactions(); + + Map vals = new HashMap<>(); + + for (int i = 0; i < TOTAL; i++) + vals.put(i, N); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + writeAllByMode(cache, vals, writeMode, INTEGER_CODEC); + + tx.commit(); + } + } + }; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + TreeSet keys = new TreeSet<>(); + + while (keys.size() < N) + keys.add(rnd.nextInt(TOTAL)); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + Map curVals = readAllByMode(cache.cache, keys, readMode, INTEGER_CODEC); + + assertEquals(N, curVals.size()); + + Map newVals = new TreeMap<>(); + + for (Map.Entry e : curVals.entrySet()) + newVals.put(e.getKey(), e.getValue() + 1); + + writeAllByMode(cache.cache, newVals, writeMode, INTEGER_CODEC); + + tx.commit(); + } + catch (Exception e) { + handleTxException(e); + } + finally { + cache.readUnlock(); + } + + cnt++; + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Set keys = new LinkedHashSet<>(); + + while (!stop.get()) { + while (keys.size() < TOTAL) + keys.add(rnd.nextInt(TOTAL)); + + TestCache cache = randomCache(caches, rnd); + + Map vals = null; + + try { + vals = readAllByMode(cache.cache, keys, readMode, INTEGER_CODEC); + } + catch (Exception e) { + handleTxException(e); + } + finally { + cache.readUnlock(); + } + + assertEquals("vals=" + vals, TOTAL, vals.size()); + + int sum = 0; + + for (int i = 0; i < TOTAL; i++) { + Integer val = vals.get(i); + + assertNotNull(val); + + sum += val; + } + + assertEquals(0, sum % N); + } + + if (idx == 0) { + TestCache cache = randomCache(caches, rnd); + + Map vals; + + try { + vals = readAllByMode(cache.cache, keys, readMode, INTEGER_CODEC); + } + finally { + cache.readUnlock(); + } + + int sum = 0; + + for (int i = 0; i < TOTAL; i++) { + Integer val = vals.get(i); + + info("Value [id=" + i + ", val=" + val + ']'); + + sum += val; + } + + info("Sum [sum=" + sum + ", mod=" + sum % N + ']'); + } + } + }; + + readWriteTest( + restartMode, + srvs, + clients, + cacheBackups, + cacheParts, + writers, + readers, + time, + cfgC, + init, + writer, + reader); + } + + /** + * @param restartMode Restart mode. + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param time Test time. + * @param cfgC Optional closure applied to cache configuration. + * @param writers Number of writers. + * @param readers Number of readers. + * @param init Optional init closure. + * @param writer Writers threads closure. + * @param reader Readers threads closure. + * @throws Exception If failed. + */ + final void readWriteTest( + final RestartMode restartMode, + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + final int writers, + final int readers, + final long time, + @Nullable IgniteInClosure cfgC, + IgniteInClosure> init, + final GridInClosure3, AtomicBoolean> writer, + final GridInClosure3, AtomicBoolean> reader) throws Exception { + if (restartMode == RestartMode.RESTART_CRD) + MvccProcessorImpl.coordinatorAssignClosure(new CoordinatorAssignClosure()); + + Ignite srv0 = startGridsMultiThreaded(srvs); + + if (clients > 0) { + client = true; + + startGridsMultiThreaded(srvs, clients); + + client = false; + } + + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), + FULL_SYNC, + cacheBackups, + cacheParts); + + if (restartMode == RestartMode.RESTART_CRD) + ccfg.setNodeFilter(new CoordinatorNodeFilter()); + + if (cfgC != null) + cfgC.apply(ccfg); + + IgniteCache cache = srv0.createCache(ccfg); + + int crdIdx = srvs + clients; + + if (restartMode == RestartMode.RESTART_CRD) { + nodeAttr = CRD_ATTR; + + startGrid(crdIdx); + } + + if (init != null) + init.apply(cache); + + final List caches = new ArrayList<>(srvs + clients); + + for (int i = 0; i < srvs + clients; i++) { + Ignite node = grid(i); + + caches.add(new TestCache(node.cache(cache.getName()))); + } + + final long stopTime = U.currentTimeMillis() + time; + + final AtomicBoolean stop = new AtomicBoolean(); + + try { + final AtomicInteger writerIdx = new AtomicInteger(); + + IgniteInternalFuture writeFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + try { + int idx = writerIdx.getAndIncrement(); + + writer.apply(idx, caches, stop); + } + catch (Throwable e) { + if (restartMode != null && X.hasCause(e, ClusterTopologyException.class)) { + log.info("Writer error: " + e); + + return null; + } + + error("Unexpected error: " + e, e); + + stop.set(true); + + fail("Unexpected error: " + e); + } + + return null; + } + }, writers, "writer"); + + final AtomicInteger readerIdx = new AtomicInteger(); + + IgniteInternalFuture readFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + try { + int idx = readerIdx.getAndIncrement(); + + reader.apply(idx, caches, stop); + } + catch (Throwable e) { + error("Unexpected error: " + e, e); + + stop.set(true); + + fail("Unexpected error: " + e); + } + + return null; + } + }, readers, "reader"); + + while (System.currentTimeMillis() < stopTime && !stop.get()) { + Thread.sleep(1000); + + if (restartMode != null) { + switch (restartMode) { + case RESTART_CRD: { + log.info("Start new coordinator: " + (crdIdx + 1)); + + startGrid(crdIdx + 1); + + log.info("Stop current coordinator: " + crdIdx); + + stopGrid(crdIdx); + + crdIdx++; + + awaitPartitionMapExchange(); + + break; + } + + case RESTART_RND_SRV: { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int idx = rnd.nextInt(srvs); + + TestCache cache0 = caches.get(idx); + + cache0.stopLock.writeLock().lock(); + + log.info("Stop node: " + idx); + + stopGrid(idx); + + log.info("Start new node: " + idx); + + Ignite srv = startGrid(idx); + + synchronized (caches) { + caches.set(idx, new TestCache(srv.cache(DEFAULT_CACHE_NAME))); + } + + awaitPartitionMapExchange(); + + break; + } + + default: + fail(); + } + } + } + + stop.set(true); + + Exception ex = null; + + try { + writeFut.get(); + } + catch (IgniteCheckedException e) { + ex = e; + } + + try { + readFut.get(); + } + catch (IgniteCheckedException e) { + if (ex != null) + ex.addSuppressed(e); + else + ex = e; + } + + if (ex != null) + throw ex; + } + finally { + stop.set(true); + } + } + + /** + * @param cacheMode Cache mode. + * @param syncMode Write synchronization mode. + * @param backups Number of backups. + * @param parts Number of partitions. + * @return Cache configuration. + */ + final CacheConfiguration cacheConfiguration( + CacheMode cacheMode, + CacheWriteSynchronizationMode syncMode, + int backups, + int parts) { + CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); + + ccfg.setCacheMode(cacheMode); + ccfg.setAtomicityMode(TRANSACTIONAL); + ccfg.setWriteSynchronizationMode(syncMode); + ccfg.setAffinity(new RendezvousAffinityFunction(false, parts)); + + if (cacheMode == PARTITIONED) + ccfg.setBackups(backups); + + return ccfg; + } + + /** + * Handles transaction exception. + * @param e Exception. + */ + protected void handleTxException(Exception e) { + if (log.isTraceEnabled()) + log.trace("Exception during tx execution: " + X.getFullStackTrace(e)); + } + + /** + * @throws Exception If failed. + */ + final void verifyCoordinatorInternalState() throws Exception { + for (Ignite node : G.allGrids()) { + final MvccProcessorImpl crd = mvccProcessor(node); + + if (crd == null) + continue; + + crd.stopVacuumWorkers(); // to prevent new futures creation. + + Map activeTxs = GridTestUtils.getFieldValue(crd, "activeTxs"); + Map cntrFuts = GridTestUtils.getFieldValue(crd, "snapLsnrs"); + Map ackFuts = GridTestUtils.getFieldValue(crd, "ackFuts"); + Map activeTrackers = GridTestUtils.getFieldValue(crd, "activeTrackers"); + + GridAbsPredicate cond = () -> { + log.info("activeTxs=" + activeTxs + ", cntrFuts=" + cntrFuts + ", ackFuts=" + ackFuts + + ", activeTrackers=" + activeTrackers); + + boolean empty = true; + + for (Map map : cntrFuts.values()) + if (!(empty = map.isEmpty())) + break; + + return activeTxs.isEmpty() && empty && ackFuts.isEmpty() && activeTrackers.isEmpty(); + }; + + GridTestUtils.waitForCondition(cond, TX_TIMEOUT); + + assertTrue("activeTxs: " + activeTxs, activeTxs.isEmpty()); + + boolean empty = true; + + for (Map map : cntrFuts.values()) + if (!(empty = map.isEmpty())) break; + + assertTrue("cntrFuts: " + cntrFuts, empty); + assertTrue("ackFuts: " + ackFuts, ackFuts.isEmpty()); + assertTrue("activeTrackers: " + activeTrackers, activeTrackers.isEmpty()); + + checkActiveQueriesCleanup(node); + } + } + + /** + * Checks if less than 2 versions remain after the vacuum cleanup. + * + * @throws Exception If failed. + */ + private void verifyOldVersionsCleaned() throws Exception { + runVacuumSync(); + + // Check versions. + boolean cleaned = checkOldVersions(false); + + if (!cleaned) { // Retry on a stable topology with a newer snapshot. + awaitPartitionMapExchange(); + + runVacuumSync(); + + checkOldVersions(true); + } + } + + /** + * Checks if outdated versions were cleaned after the vacuum process. + * + * @param failIfNotCleaned Fail test if not cleaned. + * @return {@code False} if not cleaned. + * @throws IgniteCheckedException If failed. + */ + private boolean checkOldVersions(boolean failIfNotCleaned) throws IgniteCheckedException { + for (Ignite node : G.allGrids()) { + for (IgniteCacheProxy cache : ((IgniteKernal)node).caches()) { + GridCacheContext cctx = cache.context(); + + if (!cctx.userCache() || !cctx.group().mvccEnabled()) + continue; + + for (Object e : cache.withKeepBinary()) { + IgniteBiTuple entry = (IgniteBiTuple)e; + + KeyCacheObject key = cctx.toCacheKeyObject(entry.getKey()); + + List> vers = cctx.offheap().mvccAllVersions(cctx, key) + .stream().filter(t -> t.get1() != null).collect(Collectors.toList()); + + if (vers.size() > 1) { + if (failIfNotCleaned) + fail("[key=" + key.value(null, false) + "; vers=" + vers + ']'); + else + return false; + } + } + } + } + + return true; + } + + /** + * Runs vacuum on all nodes and waits for its completion. + * + * @throws IgniteCheckedException If failed. + */ + private void runVacuumSync() throws IgniteCheckedException { + GridCompoundIdentityFuture fut = new GridCompoundIdentityFuture<>(); + + // Run vacuum manually. + for (Ignite node : G.allGrids()) { + if (!node.configuration().isClientMode()) { + MvccProcessorImpl crd = mvccProcessor(node); + + if (crd == null) + continue; + + Throwable vacuumError = crd.vacuumError(); + + assertNull(X.getFullStackTrace(vacuumError), vacuumError); + + fut.add(crd.runVacuum()); + } + } + + fut.markInitialized(); + + // Wait vacuum finished. + fut.get(); + } + + /** + * @param node Ignite node. + * @return Mvcc processor. + */ + protected MvccProcessorImpl mvccProcessor(Ignite node) { + GridKernalContext ctx = ((IgniteEx)node).context(); + + MvccProcessor crd = ctx.coordinators(); + + assertNotNull(crd); + + if (crd instanceof NoOpMvccProcessor) { + assertFalse(MvccUtils.mvccEnabled(ctx)); + + return null; + } + + return (MvccProcessorImpl)crd; + } + + /** + * @param node Node. + * @throws Exception If failed. + */ + protected final void checkActiveQueriesCleanup(Ignite node) throws Exception { + final MvccProcessorImpl crd = mvccProcessor(node); + + assertTrue("Active queries not cleared: " + node.name(), GridTestUtils.waitForCondition( + new GridAbsPredicate() { + @Override public boolean apply() { + Object activeQueries = GridTestUtils.getFieldValue(crd, "activeQueries"); + + synchronized (activeQueries) { + Long minQry = GridTestUtils.getFieldValue(activeQueries, "minQry"); + + if (minQry != null) + log.info("Min query: " + minQry); + + Map queriesMap = GridTestUtils.getFieldValue(activeQueries, "activeQueries"); + + boolean empty = true; + + for (Map.Entry e : queriesMap.entrySet()) { + if (!e.getValue().isEmpty()) { + empty = false; + + log.info("Active queries: " + e); + } + } + + return empty && minQry == null; + } + } + }, 8_000) + ); + + assertTrue("Previous coordinator queries not empty: " + node.name(), GridTestUtils.waitForCondition( + new GridAbsPredicate() { + @Override public boolean apply() { + Map queries = GridTestUtils.getFieldValue(crd, "prevCrdQueries", "activeQueries"); + Boolean prevDone = GridTestUtils.getFieldValue(crd, "prevCrdQueries", "prevQueriesDone"); + + if (!queries.isEmpty() || !prevDone) + log.info("Previous coordinator state [prevDone=" + prevDone + ", queries=" + queries + ']'); + + return queries.isEmpty(); + } + }, 8_000) + ); + } + + /** + * @return Cache configurations. + */ + protected List> cacheConfigurations() { + List> ccfgs = new ArrayList<>(); + + ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, RendezvousAffinityFunction.DFLT_PARTITION_COUNT)); + ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, RendezvousAffinityFunction.DFLT_PARTITION_COUNT)); + ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 2, RendezvousAffinityFunction.DFLT_PARTITION_COUNT)); + ccfgs.add(cacheConfiguration(REPLICATED, FULL_SYNC, 0, RendezvousAffinityFunction.DFLT_PARTITION_COUNT)); + + return ccfgs; + } + + /** + * Reads value from cache for the given key using given read mode. + * + * @param cache Cache. + * @param key Key. + * @param readMode Read mode. + * @param codec Sql object codec. + * @return Value. + */ + @SuppressWarnings("unchecked") + protected Object readByMode(IgniteCache cache, final Object key, ReadMode readMode, ObjectCodec codec) { + assert cache != null && key != null && readMode != null && readMode != SQL_SUM; + assert readMode != SQL || codec != null; + + boolean emulateLongQry = ThreadLocalRandom.current().nextBoolean(); + + switch (readMode) { + case GET: + return cache.get(key); + + case SCAN: + ScanQuery scanQry = new ScanQuery(new IgniteBiPredicate() { + @Override public boolean apply(Object k, Object v) { + if (emulateLongQry) + doSleep(ThreadLocalRandom.current().nextInt(50)); + + return k.equals(key); + } + }); + + List res = cache.query(scanQry).getAll(); + + assertTrue(res.size() <= 1); + + return res.isEmpty() ? null : ((IgniteBiTuple)res.get(0)).getValue(); + + case SQL: + String qry = "SELECT * FROM " + codec.tableName() + " WHERE _key=" + key; + + SqlFieldsQuery sqlFieldsQry = new SqlFieldsQuery(qry); + + if (emulateLongQry) + sqlFieldsQry.setLazy(true).setPageSize(1); + + List rows; + + if (emulateLongQry) { + FieldsQueryCursor cur = cache.query(sqlFieldsQry); + + rows = new ArrayList<>(); + + for (List row : cur) { + rows.add(row); + + doSleep(ThreadLocalRandom.current().nextInt(50)); + } + } + else + rows = cache.query(sqlFieldsQry).getAll(); + + assertTrue(rows.size() <= 1); + + return rows.isEmpty() ? null : codec.decode(rows.get(0)); + + default: + throw new AssertionError("Unsupported read mode: " + readMode); + } + } + + /** + * Writes value into cache using given write mode. + * + * @param cache Cache. + * @param key Key. + * @param val Value. + * @param writeMode Write mode. + * @param codec Sql object codec. + */ + @SuppressWarnings("unchecked") + protected void writeByMode(IgniteCache cache, final Object key, Object val, WriteMode writeMode, ObjectCodec codec) { + assert writeMode != DML || codec != null; + assert cache != null && key != null && writeMode != null && val != null; + + switch (writeMode) { + case PUT: + cache.put(key, val); + + return; + + case DML: + String qry = "MERGE INTO " + codec.tableName() + " (" + codec.columnsNames() + ") VALUES " + + '(' + key + ", " + codec.encode(val) + ')'; + + List rows = cache.query(new SqlFieldsQuery(qry)).getAll(); + + assertTrue(rows.size() <= 1); + + return; + + default: + throw new AssertionError("Unsupported write mode: " + writeMode); + } + } + + + /** + * Reads value from cache for the given key using given read mode. + * + * @param cache Cache. + * @param keys Key. + * @param readMode Read mode. + * @param codec Value codec. + * @return Value. + */ + @SuppressWarnings("unchecked") + protected Map readAllByMode(IgniteCache cache, Set keys, ReadMode readMode, ObjectCodec codec) { + assert cache != null && keys != null && readMode != null; + assert readMode != SQL || codec != null; + + boolean emulateLongQry = ThreadLocalRandom.current().nextBoolean(); + + switch (readMode) { + case GET: + return cache.getAll(keys); + + case SCAN: + ScanQuery scanQry = new ScanQuery(new IgniteBiPredicate() { + @Override public boolean apply(Object k, Object v) { + if (emulateLongQry) + doSleep(ThreadLocalRandom.current().nextInt(50)); + + return keys.contains(k); + } + }); + + + Map res = (Map)cache.query(scanQry).getAll() + .stream() + .collect(Collectors.toMap(v -> ((IgniteBiTuple)v).getKey(), v -> ((IgniteBiTuple)v).getValue())); + + assertTrue("res.size()=" + res.size() + ", keys.size()=" + keys.size(), res.size() <= keys.size()); + + return res; + + case SQL: + StringBuilder b = new StringBuilder("SELECT " + codec.columnsNames() + " FROM " + codec.tableName() + " WHERE _key IN ("); + + boolean first = true; + + for (Object key : keys) { + if (first) + first = false; + else + b.append(", "); + + b.append(key); + } + + b.append(')'); + + String qry = b.toString(); + + SqlFieldsQuery sqlFieldsQry = new SqlFieldsQuery(qry); + + if (emulateLongQry) + sqlFieldsQry.setLazy(true).setPageSize(1); + + List rows; + + if (emulateLongQry) { + FieldsQueryCursor cur = cache.query(sqlFieldsQry); + + rows = new ArrayList<>(); + + for (List row : cur) { + rows.add(row); + + doSleep(ThreadLocalRandom.current().nextInt(50)); + } + } + else + rows = cache.query(sqlFieldsQry).getAll(); + + if (rows.isEmpty()) + return Collections.EMPTY_MAP; + + res = new HashMap(); + + for (List row : rows) + res.put(row.get(0), codec.decode(row)); + + return res; + + case SQL_SUM: + b = new StringBuilder("SELECT SUM(" + codec.aggregateColumnName() + ") FROM " + codec.tableName() + " WHERE _key IN ("); + + first = true; + + for (Object key : keys) { + if (first) + first = false; + else + b.append(", "); + + b.append(key); + } + + b.append(')'); + + qry = b.toString(); + + FieldsQueryCursor cur = cache.query(new SqlFieldsQuery(qry)); + + rows = cur.getAll(); + + if (rows.isEmpty()) + return Collections.EMPTY_MAP; + + res = new HashMap(); + + for (List row : rows) + res.put(row.get(0), row.get(0)); + + return res; + + default: + throw new AssertionError("Unsupported read mode: " + readMode); + } + } + + /** + * Writes all entries using given write mode. + * + * @param cache Cache. + * @param entries Entries to write. + * @param writeMode Write mode. + * @param codec Entry codec. + */ + @SuppressWarnings("unchecked") + protected void writeAllByMode(IgniteCache cache, final Map entries, WriteMode writeMode, ObjectCodec codec) { + assert cache != null && entries != null && writeMode != null; + assert writeMode != DML || codec != null; + + switch (writeMode) { + case PUT: + cache.putAll(entries); + + return; + + case DML: + StringBuilder b = new StringBuilder("MERGE INTO " + codec.tableName() + " (" + codec.columnsNames() + ") VALUES "); + + boolean first = true; + + for (Object entry : entries.entrySet()) { + Map.Entry e = (Map.Entry)entry; + if (first) + first = false; + else + b.append(", "); + + b.append('(') + .append(e.getKey()) + .append(", ") + .append(codec.encode(e.getValue())) + .append(')'); + } + + String qry = b.toString(); + + cache.query(new SqlFieldsQuery(qry)).getAll(); + + return; + + default: + throw new AssertionError("Unsupported write mode: " + writeMode); + } + } + + /** + * Object codec for SQL queries. + * + * @param Type. + */ + private interface ObjectCodec { + /** + * Decodes object from SQL request result. + * + * @param row SQL request result. + * @return Decoded object. + */ + T decode(List row); + + /** + * Encodes object into SQL string for INSERT clause. + * + * @param obj Object. + * @return Sql string. + */ + String encode(T obj); + + /** + * @return Table name. + */ + String tableName(); + + /** + * @return Columns names. + */ + String columnsNames(); + + /** + * @return Column for aggregate functions. + */ + String aggregateColumnName(); + } + + /** + * Codec for {@code Integer} table. + */ + private static class IntegerCodec implements ObjectCodec { + /** {@inheritDoc} */ + @Override public Integer decode(List row) { + return (Integer)row.get(1); + } + + /** {@inheritDoc} */ + @Override public String encode(Integer obj) { + return String.valueOf(obj); + } + + /** {@inheritDoc} */ + @Override public String tableName() { + return "Integer"; + } + + /** {@inheritDoc} */ + @Override public String columnsNames() { + return "_key, _val"; + } + + /** {@inheritDoc} */ + @Override public String aggregateColumnName() { + return "_val"; + } + } + + /** + * Codec for {@code MvccTestAccount} table. + */ + private static class AccountCodec implements ObjectCodec { + /** {@inheritDoc} */ + @Override public MvccTestAccount decode(List row) { + Integer val = (Integer)row.get(1); + Integer updateCnt = (Integer)row.get(2); + + return new MvccTestAccount(val, updateCnt); + } + + /** {@inheritDoc} */ + @Override public String encode(MvccTestAccount obj) { + return String.valueOf(obj.val) + ", " + String.valueOf(obj.updateCnt); + } + + /** {@inheritDoc} */ + @Override public String tableName() { + return "MvccTestAccount"; + } + + /** {@inheritDoc} */ + @Override public String columnsNames() { + return "_key, val, updateCnt"; + } + + /** {@inheritDoc} */ + @Override public String aggregateColumnName() { + return "val"; + } + } + + + /** + * @param caches Caches. + * @param rnd Random. + * @return Random cache. + */ + static TestCache randomCache( + List caches, + ThreadLocalRandom rnd) { + synchronized (caches) { + if (caches.size() == 1) { + TestCache cache = caches.get(0); + + assertTrue(cache.readLock()); + + return cache; + } + + for (;;) { + int idx = rnd.nextInt(caches.size()); + + TestCache testCache = caches.get(idx); + + if (testCache.readLock()) + return testCache; + } + } + } + + /** + * + */ + static class MvccTestAccount { + /** */ + @QuerySqlField(index = false) + final int val; + + /** */ + @QuerySqlField + final int updateCnt; + + /** + * @param val Value. + * @param updateCnt Updates counter. + */ + MvccTestAccount(int val, int updateCnt) { + assert updateCnt > 0; + + this.val = val; + this.updateCnt = updateCnt; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "MvccTestAccount{" + + "val=" + val + + ", updateCnt=" + updateCnt + + '}'; + } + } + + /** + * + */ + enum ReadMode { + /** */ + GET, + + /** */ + SCAN, + + /** */ + SQL, + + /** */ + SQL_SUM + } + + /** + * + */ + enum WriteMode { + /** */ + DML, + + /** */ + PUT + } + + /** + * + */ + enum RestartMode { + /** + * Dedicated coordinator node is restarted during test. + */ + RESTART_CRD, + + /** */ + RESTART_RND_SRV + } + + /** + * + */ + static class CoordinatorNodeFilter implements IgnitePredicate { + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode node) { + return node.attribute(CRD_ATTR) == null; + } + } + + /** + * + */ + static class CoordinatorAssignClosure implements IgniteClosure, ClusterNode> { + /** {@inheritDoc} */ + @Override public ClusterNode apply(Collection clusterNodes) { + for (ClusterNode node : clusterNodes) { + if (node.attribute(CRD_ATTR) != null) { + assert !node.isClient(); + + return node; + } + } + + return null; + } + } + + /** + * + */ + static class TestCache { + /** */ + final IgniteCache cache; + + /** Locks node to avoid node restart while test operation is in progress. */ + final ReadWriteLock stopLock = new ReentrantReadWriteLock(); + + /** + * @param cache Cache. + */ + TestCache(IgniteCache cache) { + this.cache = cache; + } + + /** + * @return {@code True} if locked. + */ + boolean readLock() { + return stopLock.readLock().tryLock(); + } + + /** + * + */ + void readUnlock() { + stopLock.readLock().unlock(); + } + } + + /** + * + */ + static class InitIndexing implements IgniteInClosure { + /** */ + private final Class[] idxTypes; + + /** + * @param idxTypes Indexed types. + */ + InitIndexing(Class... idxTypes) { + this.idxTypes = idxTypes; + } + + /** {@inheritDoc} */ + @Override public void apply(CacheConfiguration cfg) { + cfg.setIndexedTypes(idxTypes); + } + } + + /** + * Removed accounts tracker. + */ + private static class RemovedAccountsTracker { + /** */ + private final Map rmvdKeys; + + /** + * @param size Size. + */ + RemovedAccountsTracker(int size) { + this.rmvdKeys = new HashMap<>(size); + + for (int i = 0; i < size; i++) + rmvdKeys.put(i, 0); + } + + /** + * @return Size. + */ + public synchronized int size() { + int size = 0; + + for (int i = 0; i < rmvdKeys.size(); i++) { + if (rmvdKeys.get(i) > 0) + size++; + } + + return size; + } + + /** + * @param id Id. + * @return {@code True} if success. + */ + synchronized boolean markRemoved(Integer id) { + Integer rmvdCntr = rmvdKeys.get(id); + + Integer newCntr = rmvdCntr + 1; + + rmvdKeys.put(id, newCntr); + + return newCntr >= 0; + } + + /** + * @param id Id. + * @return {@code True} if success. + */ + synchronized boolean unmarkRemoved(Integer id) { + Integer rmvdCntr = rmvdKeys.get(id); + + Integer newCntr = rmvdCntr - 1; + + rmvdKeys.put(id, newCntr); + + return newCntr >= 0; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java new file mode 100644 index 0000000000000..f9ab7bcfb208a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.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.mvcc; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.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 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; + +/** + * + */ +public class CacheMvccClusterRestartTest 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); + + cfg.setConsistentId(gridName); + + cfg.setMvccEnabled(true); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + DataStorageConfiguration storageCfg = new DataStorageConfiguration(); + + storageCfg.setWalMode(WALMode.LOG_ONLY); + storageCfg.setPageSize(1024); + + DataRegionConfiguration regionCfg = new DataRegionConfiguration(); + + regionCfg.setPersistenceEnabled(true); + regionCfg.setMaxSize(100 * 1024 * 1024); + + storageCfg.setDefaultDataRegionConfiguration(regionCfg); + + cfg.setDataStorageConfiguration(storageCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + cleanPersistenceDir(); + + super.afterTestsStopped(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + cleanPersistenceDir(); + + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If failed. + */ + public void testRestart1() throws Exception { + restart1(3, 3); + } + + /** + * @throws Exception If failed. + */ + public void testRestart2() throws Exception { + restart1(1, 3); + } + + /** + * @throws Exception If failed. + */ + public void testRestart3() throws Exception { + restart1(3, 1); + } + + /** + * @param srvBefore Number of servers before restart. + * @param srvAfter Number of servers after restart. + * @throws Exception If failed. + */ + private void restart1(int srvBefore, int srvAfter) throws Exception { + Ignite srv0 = startGridsMultiThreaded(srvBefore); + + srv0.active(true); + + IgniteCache cache = srv0.createCache(cacheConfiguration()); + + Set keys = new HashSet<>(primaryKeys(cache, 1, 0)); + + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (Integer k : keys) + cache.put(k, k); + + tx.commit(); + } + + stopAllGrids(); + + srv0 = startGridsMultiThreaded(srvAfter); + + srv0.active(true); + + cache = srv0.cache(DEFAULT_CACHE_NAME); + + Map res = cache.getAll(keys); + + assertEquals(keys.size(), res.size()); + + for (Integer k : keys) + assertEquals(k, cache.get(k)); + + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (Integer k : keys) + cache.put(k, k + 1); + + tx.commit(); + } + + for (Integer k : keys) + assertEquals(k + 1, cache.get(k)); + } + + /** + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration() { + CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); + + ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setAtomicityMode(TRANSACTIONAL); + ccfg.setBackups(2); + + return ccfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java new file mode 100644 index 0000000000000..b0a2ec84514b9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java @@ -0,0 +1,191 @@ +/* + * 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.mvcc; + +import java.util.concurrent.Callable; +import javax.cache.CacheException; +import javax.cache.configuration.Factory; +import javax.cache.expiry.CreatedExpiryPolicy; +import javax.cache.expiry.Duration; +import javax.cache.expiry.ExpiryPolicy; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheInterceptorAdapter; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.configuration.CacheConfiguration; +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; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; + +/** + * + */ +public class CacheMvccConfigurationValidationTest 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); + + cfg.setMvccEnabled(true); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If failed. + */ + public void testMvccModeMismatchForGroup1() throws Exception { + final Ignite node = startGrid(0); + + node.createCache(new CacheConfiguration("cache1").setGroupName("grp1").setAtomicityMode(ATOMIC)); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + node.createCache(new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL)); + + return null; + } + }, CacheException.class, null); + + node.createCache(new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(ATOMIC)); + } + + /** + * @throws Exception If failed. + */ + public void testMvccModeMismatchForGroup2() throws Exception { + final Ignite node = startGrid(0); + + node.createCache(new CacheConfiguration("cache1").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL)); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + node.createCache(new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(ATOMIC)); + + return null; + } + }, CacheException.class, null); + + node.createCache(new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL)); + } + + /** + * @throws Exception If failed. + */ + public void testTxCacheWithCacheStore() throws Exception { + checkTransactionalModeConflict("cacheStoreFactory", new TestFactory(), + "Transactional cache may not have a third party cache store when MVCC is enabled."); + } + + /** + * @throws Exception If failed. + */ + public void testTxCacheWithExpiryPolicy() throws Exception { + checkTransactionalModeConflict("expiryPolicyFactory0", CreatedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES), + "Transactional cache may not have expiry policy when MVCC is enabled."); + } + + /** + * @throws Exception If failed. + */ + public void testTxCacheWithInterceptor() throws Exception { + checkTransactionalModeConflict("interceptor", new CacheInterceptorAdapter(), + "Transactional cache may not have an interceptor when MVCC is enabled."); + } + + /** + * Check that setting specified property conflicts with transactional cache atomicity mode. + * @param propName Property name. + * @param obj Property value. + * @param errMsg Expected error message. + * @throws IgniteCheckedException if failed. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkTransactionalModeConflict(String propName, Object obj, String errMsg) + throws Exception { + final String setterName = "set" + propName.substring(0, 1).toUpperCase() + propName.substring(1); + + try (final Ignite node = startGrid(0)) { + final CacheConfiguration cfg = new TestConfiguration("cache"); + + cfg.setAtomicityMode(TRANSACTIONAL); + + U.invoke(TestConfiguration.class, cfg, setterName, obj); + + GridTestUtils.assertThrows(log, new Callable() { + @SuppressWarnings("unchecked") + @Override public Void call() { + node.getOrCreateCache(cfg); + + return null; + } + }, IgniteCheckedException.class, errMsg); + } + } + + /** + * Dummy class to overcome ambiguous method name "setExpiryPolicyFactory". + */ + private final static class TestConfiguration extends CacheConfiguration { + /** + * + */ + TestConfiguration(String cacheName) { + super(cacheName); + } + + /** + * + */ + @SuppressWarnings("unused") + public void setExpiryPolicyFactory0(Factory plcFactory) { + super.setExpiryPolicyFactory(plcFactory); + } + } + + /** + * + */ + private static class TestFactory implements Factory { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public CacheStore create() { + return null; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentTransactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentTransactionTest.java new file mode 100644 index 0000000000000..90c5b6e737034 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentTransactionTest.java @@ -0,0 +1,64 @@ +/* + * 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.mvcc; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.lang.IgniteClosure2X; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +public class CacheMvccIteratorWithConcurrentTransactionTest extends CacheMvccAbstractFeatureTest { + /** + * @throws Exception if failed. + */ + public void testScanQuery() throws Exception { + doTestConsistency(clo); + } + + /** Test closure. */ + private final IgniteClosure2X> clo = + new IgniteClosure2X>() { + @Override public List applyx(CountDownLatch startLatch, CountDownLatch endLatch2) + throws IgniteCheckedException { + Iterator> it = cache().iterator(); + + List> pres = new ArrayList<>(); + + for (int i = 0; i < 50; i++) + pres.add(it.next()); + + if (startLatch != null) + startLatch.countDown(); + + while (it.hasNext()) + pres.add(it.next()); + + if (endLatch2 != null) + U.await(endLatch2); + + return entriesToPersons(pres); + } + }; +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentTransactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentTransactionTest.java new file mode 100644 index 0000000000000..f4c9781744f05 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentTransactionTest.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.internal.util.lang.IgniteClosure2X; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +public class CacheMvccLocalEntriesWithConcurrentTransactionTest extends CacheMvccAbstractFeatureTest { + /** + * @throws Exception if failed. + */ + public void testLocalEntries() throws Exception { + doTestConsistency(clo); + } + + /** Test closure. */ + private final IgniteClosure2X> clo = + new IgniteClosure2X>() { + @Override public List applyx(CountDownLatch startLatch, CountDownLatch endLatch2) + throws IgniteCheckedException { + Iterator> it = cache().localEntries(CachePeekMode.PRIMARY).iterator(); + + List> pres = new ArrayList<>(); + + for (int i = 0; i < 10; i++) + pres.add(it.next()); + + if (startLatch != null) + startLatch.countDown(); + + while (it.hasNext()) + pres.add(it.next()); + + if (endLatch2 != null) + U.await(endLatch2); + + return entriesToPersons(pres); + } + }; +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccOperationChecksTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccOperationChecksTest.java new file mode 100644 index 0000000000000..98a06c7f5092e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccOperationChecksTest.java @@ -0,0 +1,201 @@ +/* + * 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.mvcc; + +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.Callable; +import javax.cache.expiry.EternalExpiryPolicy; +import javax.cache.expiry.ExpiryPolicy; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.cache.GridCacheAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.testframework.GridTestUtils; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * + */ +public class CacheMvccOperationChecksTest extends CacheMvccAbstractTest { + /** Empty Class[]. */ + private final static Class[] E = new Class[]{}; + + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + } + + /** + * @throws Exception if failed. + */ + public void testClearOperationsUnsupported() throws Exception { + checkOperationUnsupported("clear", m("Clear"), E); + + checkOperationUnsupported("clearAsync", m("Clear"), E); + + checkOperationUnsupported("clear", m("Clear"), t(Object.class), 1); + + checkOperationUnsupported("clearAsync", m("Clear"), t(Object.class), 1); + + checkOperationUnsupported("clearAll", m("Clear"), t(Set.class), Collections.singleton(1)); + + checkOperationUnsupported("clearAllAsync", m("Clear"), t(Set.class), + Collections.singleton(1)); + } + + /** + * @throws Exception if failed. + */ + public void testLoadOperationsUnsupported() throws Exception { + checkOperationUnsupported("loadCache", m("Load"), t(IgniteBiPredicate.class, Object[].class), + P, new Object[]{ 1 }); + + checkOperationUnsupported("loadCacheAsync", m("Load"), t(IgniteBiPredicate.class, Object[].class), + P, new Object[]{ 1 }); + + checkOperationUnsupported("localLoadCache", m("Load"), t(IgniteBiPredicate.class, Object[].class), + P, new Object[]{ 1 }); + + checkOperationUnsupported("localLoadCacheAsync", m("Load"), t(IgniteBiPredicate.class, Object[].class), + P, new Object[]{ 1 }); + } + + /** + * @throws Exception if failed. + */ + public void testLockOperationsUnsupported() throws Exception { + checkOperationUnsupported("lock", m("Lock"), t(Object.class), 1); + + checkOperationUnsupported("lockAll", m("Lock"), t(Collection.class), Collections.singleton(1)); + } + + /** + * @throws Exception if failed. + */ + public void testPeekOperationsUnsupported() throws Exception { + checkOperationUnsupported("localPeek", m("Peek"), t(Object.class, CachePeekMode[].class), 1, + new CachePeekMode[]{CachePeekMode.NEAR}); + } + + /** + * @throws Exception if failed. + */ + public void testEvictOperationsUnsupported() throws Exception { + checkOperationUnsupported("localEvict", m("Evict"), t(Collection.class), Collections.singleton(1)); + } + + /** + * @throws Exception if failed. + */ + public void testWithExpiryPolicyUnsupported() throws Exception { + checkOperationUnsupported("withExpiryPolicy", m("withExpiryPolicy"), t(ExpiryPolicy.class), + EternalExpiryPolicy.factoryOf().create()); + } + + /** + * @param opTypeName Operation type name. + * @return Typical error message from {@link GridCacheAdapter}. + */ + private static String m(String opTypeName) { + return opTypeName + " operations are not supported on transactional caches when MVCC is enabled."; + } + + /** + * @param types Parameter types. + * @return Types array. + */ + private static Class[] t(Class... types) { + return types; + } + + /** + * @param mtdName Method name. + * @param errMsg Expected error message. + * @param paramTypes Operation param types. + * @param args Operation arguments. + * @throws Exception if failed. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkOperationUnsupported(String mtdName, String errMsg, Class[] paramTypes, + Object... args) throws Exception { + final boolean async = mtdName.endsWith("Async"); + + try (final Ignite node = startGrid(0)) { + final CacheConfiguration cfg = new CacheConfiguration<>("cache"); + + cfg.setCacheMode(cacheMode()); + cfg.setAtomicityMode(TRANSACTIONAL); + + try (IgniteCache cache = node.createCache(cfg)) { + GridTestUtils.assertThrows(log, new Callable() { + @SuppressWarnings("unchecked") + @Override public Void call() throws Exception { + try { + Object o = U.invoke(null, cache, mtdName, paramTypes, args); + + if (async) { + assertTrue(o instanceof IgniteFuture); + + ((IgniteFuture)o).get(); + } + } + catch (Exception e) { + if (e.getCause() == null) + throw e; + + if (e.getCause().getCause() == null) + throw e; + + throw (Exception)e.getCause().getCause(); + } + + return null; + } + }, UnsupportedOperationException.class, errMsg); + } + } + } + + /** + * + */ + private final static IgniteBiPredicate P = new IgniteBiPredicate() { + @Override public boolean apply(Object o, Object o2) { + return false; + } + }; +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedCoordinatorFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedCoordinatorFailoverTest.java new file mode 100644 index 0000000000000..3ea1c5bb35317 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedCoordinatorFailoverTest.java @@ -0,0 +1,144 @@ +/* + * 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.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.GET; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SCAN; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.PUT; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Coordinator failover test for partitioned caches. + */ +public class CacheMvccPartitionedCoordinatorFailoverTest extends CacheMvccAbstractCoordinatorFailoverTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGet_ClientServer_Backups2_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + accountsTxReadAll(4, 2, 2, DFLT_PARTITION_COUNT, + null, true, GET, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGet_Server_Backups1_CoordinatorFails() throws Exception { + accountsTxReadAll(2, 0, 1, DFLT_PARTITION_COUNT, + null, true, GET, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_ClientServer_Backups2_CoordinatorFails() throws Exception { + accountsTxReadAll(4, 2, 2, DFLT_PARTITION_COUNT, + null, true, SCAN, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_Server_Backups1_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + accountsTxReadAll(2, 0, 1, DFLT_PARTITION_COUNT, + null, true, SCAN, PUT, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups2_RestartCoordinator_GetPut() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD, 4, 2, 2, DFLT_PARTITION_COUNT, + null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_RestartCoordinator_GetPut_Persistence() throws Exception { + persistence = true; + + putAllGetAll(RestartMode.RESTART_CRD, 2, 1, 1, 64, + null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_ClientServer_Backups1_PutGet_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + updateNObjectsTest(3, 5, 3, 1, DFLT_PARTITION_COUNT, DFLT_TEST_TIME, + null, GET, PUT, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_ClientServer_Backups1__PutGet_CoordinatorFails() throws Exception { + updateNObjectsTest(10, 3, 2, 1, DFLT_PARTITION_COUNT, DFLT_TEST_TIME, + null, GET, PUT, RestartMode.RESTART_CRD); + } + + + /** + * @throws Exception If failed. + */ + public void testGetReadInProgressCoordinatorFails() throws Exception { + readInProgressCoordinatorFails(false, false, PESSIMISTIC, REPEATABLE_READ, GET, PUT, null); + } + + /** + * @throws Exception If failed. + */ + public void testGetReadInsideTxInProgressCoordinatorFails() throws Exception { + readInProgressCoordinatorFails(false, true, PESSIMISTIC, REPEATABLE_READ, GET, PUT, null); + } + + /** + * @throws Exception If failed. + */ + public void testGetReadInProgressCoordinatorFails_ReadDelay() throws Exception { + readInProgressCoordinatorFails(true, false, PESSIMISTIC, REPEATABLE_READ, GET, PUT, null); + } + + /** + * @throws Exception If failed. + */ + public void testGetReadInsideTxInProgressCoordinatorFails_ReadDelay() throws Exception { + readInProgressCoordinatorFails(true, true, PESSIMISTIC, REPEATABLE_READ, GET, PUT, null); + } + + /** + * @throws Exception If failed. + */ + public void testReadInProgressCoordinatorFailsSimple_FromServerPutGet() throws Exception { + readInProgressCoordinatorFailsSimple(false, null, GET, PUT); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccProcessorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccProcessorTest.java new file mode 100644 index 0000000000000..411a4b74ed4c1 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccProcessorTest.java @@ -0,0 +1,97 @@ +/* + * 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.mvcc; + +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * + */ +public class CacheMvccProcessorTest extends CacheMvccAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return PARTITIONED; + } + + /** + * @throws Exception If failed. + */ + public void testTreeWithPersistence() throws Exception { + persistence = true; + + checkTreeOperations(); + } + + /** + * @throws Exception If failed. + */ + public void testTreeWithoutPersistence() throws Exception { + persistence = true; + + checkTreeOperations(); + } + + /** + * @throws Exception If failed. + */ + private void checkTreeOperations() throws Exception { + IgniteEx grid = startGrid(0); + + grid.cluster().active(true); + + MvccProcessorImpl mvccProcessor = mvccProcessor(grid); + + assertEquals(TxState.NA, mvccProcessor.state(new MvccVersionImpl(1, 1, MvccUtils.MVCC_OP_COUNTER_NA))); + + mvccProcessor.updateState(new MvccVersionImpl(1, 1, MvccUtils.MVCC_OP_COUNTER_NA), TxState.PREPARED); + mvccProcessor.updateState(new MvccVersionImpl(1, 2, MvccUtils.MVCC_OP_COUNTER_NA), TxState.PREPARED); + mvccProcessor.updateState(new MvccVersionImpl(1, 3, MvccUtils.MVCC_OP_COUNTER_NA), TxState.COMMITTED); + mvccProcessor.updateState(new MvccVersionImpl(1, 4, MvccUtils.MVCC_OP_COUNTER_NA), TxState.ABORTED); + mvccProcessor.updateState(new MvccVersionImpl(1, 5, MvccUtils.MVCC_OP_COUNTER_NA), TxState.ABORTED); + mvccProcessor.updateState(new MvccVersionImpl(1, 6, MvccUtils.MVCC_OP_COUNTER_NA), TxState.PREPARED); + + if (persistence) { + stopGrid(0, false); + grid = startGrid(0); + + grid.cluster().active(true); + + mvccProcessor = mvccProcessor(grid); + } + + assertEquals(TxState.PREPARED, mvccProcessor.state(new MvccVersionImpl(1, 1, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.PREPARED, mvccProcessor.state(new MvccVersionImpl(1, 2, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.COMMITTED, mvccProcessor.state(new MvccVersionImpl(1, 3, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.ABORTED, mvccProcessor.state(new MvccVersionImpl(1, 4, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.ABORTED, mvccProcessor.state(new MvccVersionImpl(1, 5, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.PREPARED, mvccProcessor.state(new MvccVersionImpl(1, 6, MvccUtils.MVCC_OP_COUNTER_NA))); + + mvccProcessor.removeUntil(new MvccVersionImpl(1, 5, MvccUtils.MVCC_OP_COUNTER_NA)); + + assertEquals(TxState.NA, mvccProcessor.state(new MvccVersionImpl(1, 1, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.NA, mvccProcessor.state(new MvccVersionImpl(1, 2, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.NA, mvccProcessor.state(new MvccVersionImpl(1, 3, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.NA, mvccProcessor.state(new MvccVersionImpl(1, 4, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.NA, mvccProcessor.state(new MvccVersionImpl(1, 5, MvccUtils.MVCC_OP_COUNTER_NA))); + assertEquals(TxState.PREPARED, mvccProcessor.state(new MvccVersionImpl(1, 6, MvccUtils.MVCC_OP_COUNTER_NA))); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedCoordinatorFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedCoordinatorFailoverTest.java new file mode 100644 index 0000000000000..dc948cd9e460c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedCoordinatorFailoverTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +/** + * Coordinator failover test for replicated caches. + */ +public class CacheMvccReplicatedCoordinatorFailoverTest extends CacheMvccAbstractCoordinatorFailoverTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.REPLICATED; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentTransactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentTransactionTest.java new file mode 100644 index 0000000000000..8af6a5b9c03ab --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentTransactionTest.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.processors.cache.mvcc; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.internal.util.lang.IgniteClosure2X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; + +/** + * + */ +public class CacheMvccScanQueryWithConcurrentTransactionTest extends CacheMvccAbstractFeatureTest { + /** + * @throws Exception if failed. + */ + public void testScanQuery() throws Exception { + doTestConsistency(clo); + } + + /** Test closure. */ + private final IgniteClosure2X> clo = + new IgniteClosure2X>() { + @Override public List applyx(CountDownLatch startLatch, CountDownLatch endLatch2) + throws IgniteCheckedException { + IgniteBiPredicate f = new IgniteBiPredicate() { + @Override public boolean apply(Integer k, Person v) { + return k % 2 == 0; + } + }; + + try (QueryCursor> cur = cache().query(new ScanQuery() + .setFilter(f))) { + Iterator> it = cur.iterator(); + + List> pres = new ArrayList<>(); + + for (int i = 0; i < 50; i++) + pres.add(it.next()); + + if (startLatch != null) + startLatch.countDown(); + + while (it.hasNext()) + pres.add(it.next()); + + if (endLatch2 != null) + U.await(endLatch2); + + return entriesToPersons(pres); + } + } + }; +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentTransactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentTransactionTest.java new file mode 100644 index 0000000000000..2b8b73ed2f455 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentTransactionTest.java @@ -0,0 +1,52 @@ +/* + * 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.mvcc; + +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.lang.IgniteClosure2X; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +public class CacheMvccSizeWithConcurrentTransactionTest extends CacheMvccAbstractFeatureTest { + /** + * @throws Exception if failed. + */ + public void testSize() throws Exception { + doTestConsistency(clo); + } + + /** Test closure. */ + private final IgniteClosure2X clo = + new IgniteClosure2X() { + @Override public Integer applyx(CountDownLatch startLatch, CountDownLatch endLatch2) + throws IgniteCheckedException { + if (startLatch != null) + startLatch.countDown(); + + int res = cache().size(); + + if (endLatch2 != null) + U.await(endLatch2); + + return res; + } + }; +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java new file mode 100644 index 0000000000000..01268decca5c3 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java @@ -0,0 +1,3674 @@ +/* + * 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.mvcc; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import javax.cache.Cache; +import javax.cache.expiry.Duration; +import javax.cache.expiry.TouchedExpiryPolicy; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.cluster.ClusterTopologyException; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.TestCacheNodeExcludingFilter; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestQueryCntr; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccAckRequestTx; +import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccSnapshotResponse; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.lang.GridInClosure3; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.apache.ignite.transactions.TransactionOptimisticException; +import org.jetbrains.annotations.Nullable; +import org.junit.Assert; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.GET; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SCAN; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.PUT; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker.MVCC_TRACKER_ID_NA; +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; + +/** + * TODO IGNITE-6739: tests reload + * TODO IGNITE-6739: extend tests to use single/mutiple nodes, all tx types. + * TODO IGNITE-6739: test with cache groups. + */ +@SuppressWarnings("unchecked") +public class CacheMvccTransactionsTest extends CacheMvccAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return PARTITIONED; + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticTx1() throws Exception { + checkTx1(PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticSerializableTx1() throws Exception { + checkTx1(OPTIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticRepeatableReadTx1() throws Exception { + checkTx1(OPTIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticReadCommittedTx1() throws Exception { + checkTx1(OPTIMISTIC, READ_COMMITTED); + } + + /** + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @throws Exception If failed. + */ + private void checkTx1(final TransactionConcurrency concurrency, final TransactionIsolation isolation) + throws Exception { + checkTxWithAllCaches(new CI1>() { + @Override public void apply(IgniteCache cache) { + try { + IgniteTransactions txs = cache.unwrap(Ignite.class).transactions(); + + List keys = testKeys(cache); + + for (Integer key : keys) { + log.info("Test key: " + key); + + try (Transaction tx = txs.txStart(concurrency, isolation)) { + Integer val = cache.get(key); + + assertNull(val); + + cache.put(key, key); + + val = (Integer)checkAndGet(true, cache, key, GET, SCAN); + + assertEquals(key, val); + + tx.commit(); + } + + Integer val = (Integer)checkAndGet(false, cache, key, SCAN, GET); + + assertEquals(key, val); + } + } + catch (Exception e) { + throw new IgniteException(e); + } + } + }); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticTx2() throws Exception { + checkTx2(PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticSerializableTx2() throws Exception { + checkTx2(OPTIMISTIC, SERIALIZABLE); + } + + /** + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @throws Exception If failed. + */ + private void checkTx2(final TransactionConcurrency concurrency, final TransactionIsolation isolation) + throws Exception { + checkTxWithAllCaches(new CI1>() { + @Override public void apply(IgniteCache cache) { + try { + IgniteTransactions txs = cache.unwrap(Ignite.class).transactions(); + + List keys = testKeys(cache); + + for (Integer key : keys) { + log.info("Test key: " + key); + + try (Transaction tx = txs.txStart(concurrency, isolation)) { + cache.put(key, key); + cache.put(key + 1, key + 1); + + assertEquals(key, checkAndGet(true, cache, key, GET, SCAN)); + assertEquals(key + 1, checkAndGet(true, cache, key + 1, GET, SCAN)); + + tx.commit(); + } + + assertEquals(key, checkAndGet(false, cache, key, GET, SCAN)); + assertEquals(key + 1, checkAndGet(false, cache, key + 1, GET, SCAN)); + } + } + catch (Exception e) { + throw new IgniteException(e); + } + } + }); + } + + /** + * @param c Closure to run. + * @throws Exception If failed. + */ + private void checkTxWithAllCaches(IgniteInClosure> c) throws Exception { + client = false; + + startGridsMultiThreaded(SRVS); + + client = true; + + startGrid(SRVS); + + try { + for (CacheConfiguration ccfg : cacheConfigurations()) { + logCacheInfo(ccfg); + + ignite(0).createCache(ccfg); + + try { + Ignite node = ignite(0); + + IgniteCache cache = node.cache(ccfg.getName()); + + c.apply(cache); + } + finally { + ignite(0).destroyCache(ccfg.getName()); + } + } + + verifyCoordinatorInternalState(); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + public void testWithCacheGroups() throws Exception { + Ignite srv0 = startGrid(0); + + List ccfgs = new ArrayList<>(); + + for (int c = 0; c < 3; c++) { + CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, FULL_SYNC, 0, DFLT_PARTITION_COUNT); + + ccfg.setName("cache-" + c); + ccfg.setGroupName("grp1"); + + ccfgs.add(ccfg); + } + + srv0.createCaches(ccfgs); + + final int PUTS = 5; + + for (int i = 0; i < PUTS; i++) { + for (int c = 0; c < 3; c++) { + IgniteCache cache = srv0.cache("cache-" + c); + + Map vals = new HashMap<>(); + + for (int k = 0; k < 10; k++) { + cache.put(k, i); + + vals.put(k, i); + + assertEquals(i, checkAndGet(false, cache, k, SCAN, GET)); + } + + assertEquals(vals, checkAndGetAll(false, cache, vals.keySet(), GET, SCAN)); + } + } + + for (int c = 0; c < 3; c++) { + IgniteCache cache = srv0.cache("cache-" + c); + + Map vals = new HashMap<>(); + + for (int k = 0; k < 10; k++) { + if (k % 2 == 0) + vals.put(k, PUTS - 1); + else { + cache.remove(k); + + assertNull(checkAndGet(false, cache, k, SCAN, GET)); + } + } + + assertEquals(vals, checkAndGetAll(false, cache, vals.keySet(), GET, SCAN)); + } + } + + /** + * @throws Exception If failed. + */ + public void testCacheRecreate() throws Exception { + cacheRecreate(null); + } + + /** + * @throws Exception If failed. + */ + public void testActiveQueriesCleanup() throws Exception { + activeQueriesCleanup(false); + } + + /** + * @throws Exception If failed. + */ + public void testActiveQueriesCleanupTx() throws Exception { + activeQueriesCleanup(true); + } + + /** + * @param tx If {@code true} tests reads inside transaction. + * @throws Exception If failed. + */ + private void activeQueriesCleanup(final boolean tx) throws Exception { + startGridsMultiThreaded(SRVS); + + client = true; + + Ignite client = startGrid(SRVS); + + final int NODES = SRVS + 1; + + CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, FULL_SYNC, 1, 512); + + client.createCache(ccfg); + + final long stopTime = System.currentTimeMillis() + 5000; + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer idx) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Ignite node = ignite(idx % NODES); + + IgniteTransactions txs = node.transactions(); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + while (System.currentTimeMillis() < stopTime) { + int keyCnt = rnd.nextInt(10) + 1; + + Set keys = new HashSet<>(); + + for (int i = 0; i < keyCnt; i++) + keys.add(rnd.nextInt()); + + if (tx) { + try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { + cache.getAll(keys); + + if (rnd.nextBoolean()) + tx.commit(); + else + tx.rollback(); + } + } + else + cache.getAll(keys); + } + } + }, NODES * 2, "get-thread"); + + for (Ignite node : G.allGrids()) + checkActiveQueriesCleanup(node); + } + + /** + * @throws Exception If failed. + */ + public void testTxReadIsolationSimple() throws Exception { + Ignite srv0 = startGrids(4); + + client = true; + + startGrid(4); + + for (CacheConfiguration ccfg : cacheConfigurations()) { + IgniteCache cache0 = srv0.createCache(ccfg); + + final Map startVals = new HashMap<>(); + + final int KEYS = 10; + + for (int i = 0; i < KEYS; i++) + startVals.put(i, 0); + + for (final TransactionIsolation isolation : TransactionIsolation.values()) { + for (final Ignite node : G.allGrids()) { + info("Run test [node=" + node.name() + ", isolation=" + isolation + ']'); + + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache0.putAll(startVals); + + tx.commit(); + } + + final CountDownLatch readStart = new CountDownLatch(1); + + final CountDownLatch readProceed = new CountDownLatch(1); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = node.transactions().txStart(OPTIMISTIC, isolation)) { + assertEquals(0, checkAndGet(false, cache, 0, SCAN, GET)); + + readStart.countDown(); + + assertTrue(readProceed.await(5, TimeUnit.SECONDS)); + + if (isolation == READ_COMMITTED) { + assertNull(checkAndGet(false, cache, 1, SCAN, GET)); + + assertEquals(1, checkAndGet(false, cache, 2, SCAN, GET)); + + Map res = checkAndGetAll(false, cache, startVals.keySet(), SCAN, GET); + + assertEquals(startVals.size() / 2, res.size()); + + for (Map.Entry e : res.entrySet()) + assertEquals("Invalid value for key: " + e.getKey(), 1, e.getValue()); + } + else { + assertEquals(0, checkAndGet(true, cache, 1, GET, SCAN)); + + assertEquals(0, checkAndGet(true, cache, 2, GET, SCAN)); + + Map res = checkAndGetAll(true, cache, startVals.keySet(), GET, SCAN); + + assertEquals(startVals.size(), res.size()); + + for (Map.Entry e : res.entrySet()) + assertEquals("Invalid value for key: " + e.getKey(), 0, e.getValue()); + } + + tx.rollback(); + } + + return null; + } + }); + + assertTrue(readStart.await(5, TimeUnit.SECONDS)); + + for (int i = 0; i < KEYS; i++) { + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + if (i % 2 == 0) + cache0.put(i, 1); + else + cache0.remove(i); + + tx.commit(); + } + } + + readProceed.countDown(); + + fut.get(); + } + } + + srv0.destroyCache(cache0.getName()); + } + } + + /** + * @throws Exception If failed. + */ + public void testPutGetAllSimple() throws Exception { + Ignite node = startGrid(0); + + IgniteTransactions txs = node.transactions(); + + final IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1)); + + final int KEYS = 10_000; + + Set keys = new HashSet<>(); + + for (int k = 0; k < KEYS; k++) + keys.add(k); + + Map map = checkAndGetAll(false, cache, keys, SCAN, GET); + + assertTrue(map.isEmpty()); + + for (int v = 0; v < 3; v++) { + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = 0; k < KEYS; k++) { + if (k % 2 == 0) + cache.put(k, v); + } + + tx.commit(); + } + + map = checkAndGetAll(false, cache, keys, SCAN, GET); + + for (int k = 0; k < KEYS; k++) { + if (k % 2 == 0) + assertEquals(v, map.get(k)); + else + assertNull(map.get(k)); + } + + assertEquals(KEYS / 2, map.size()); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + map = checkAndGetAll(true, cache, keys, SCAN, GET); + + for (int k = 0; k < KEYS; k++) { + if (k % 2 == 0) + assertEquals(v, map.get(k)); + else + assertNull(map.get(k)); + } + + assertEquals(KEYS / 2, map.size()); + + tx.commit(); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testPutRemoveSimple() throws Exception { + putRemoveSimple(false); + } + + /** + * @throws Exception If failed. + */ + public void testPutRemoveSimple_LargeKeys() throws Exception { + putRemoveSimple(true); + } + + /** + * @throws Exception If failed. + * @param largeKeys {@code True} to use large keys (not fitting in single page). + */ + private void putRemoveSimple(boolean largeKeys) throws Exception { + Ignite node = startGrid(0); + + IgniteTransactions txs = node.transactions(); + + final IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1)); + + final int KEYS = 100; + + checkValues(new HashMap<>(), cache); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = 0; k < KEYS; k++) + cache.remove(testKey(largeKeys, k)); + + tx.commit(); + } + + checkValues(new HashMap<>(), cache); + + Map expVals = new HashMap<>(); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = 0; k < KEYS; k++) { + Object key = testKey(largeKeys, k); + + expVals.put(key, k); + + cache.put(key, k); + } + + tx.commit(); + } + + checkValues(expVals, cache); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = 0; k < KEYS; k++) { + if (k % 2 == 0) { + Object key = testKey(largeKeys, k); + + cache.remove(key); + + expVals.remove(key); + } + } + + tx.commit(); + } + + checkValues(expVals, cache); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Object key = testKey(largeKeys, 0); + + for (int i = 0; i < 500; i++) { + boolean rmvd; + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + if (rnd.nextBoolean()) { + cache.remove(key); + + rmvd = true; + } + else { + cache.put(key, i); + + rmvd = false; + } + + tx.commit(); + } + + if (rmvd) { + assertNull(checkAndGet(false, cache, key, SCAN, GET)); + assertTrue(checkAndGetAll(false, cache, F.asSet(key), SCAN, GET).isEmpty()); + } + else { + assertEquals(i, checkAndGet(false, cache, key, SCAN, GET)); + + Map res = checkAndGetAll(false, cache, F.asSet(key), SCAN, GET); + + assertEquals(i, res.get(key)); + } + } + } + + /** + * @param largeKeys {@code True} to use large keys (not fitting in single page). + * @param idx Index. + * @return Key instance. + */ + private static Object testKey(boolean largeKeys, int idx) { + if (largeKeys) { + int payloadSize = PAGE_SIZE + ThreadLocalRandom.current().nextInt(PAGE_SIZE * 10); + + return new TestKey(idx, payloadSize); + } + else + return idx; + } + + /** + * @param expVals Expected values. + * @param cache Cache. + */ + private void checkValues(Map expVals, IgniteCache cache) { + for (Map.Entry e : expVals.entrySet()) + assertEquals(e.getValue(), checkAndGet(false, cache, e.getKey(), SCAN, GET)); + + Map res = checkAndGetAll(false, cache, expVals.keySet(), SCAN, GET); + + assertEquals(expVals, res); + + res = new HashMap<>(); + + for (IgniteCache.Entry e : cache) + res.put(e.getKey(), e.getValue()); + + assertEquals(expVals, res); + } + + /** + * @throws Exception If failed. + */ + public void testThreadUpdatesAreVisibleForThisThread() throws Exception { + final Ignite ignite = startGrid(0); + + final IgniteCache cache = ignite.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1)); + + final int THREADS = Runtime.getRuntime().availableProcessors() * 2; + + final int KEYS = 10; + + final CyclicBarrier b = new CyclicBarrier(THREADS); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer idx) { + try { + int min = idx * KEYS; + int max = min + KEYS; + + Set keys = new HashSet<>(); + + for (int k = min; k < max; k++) + keys.add(k); + + b.await(); + + for (int i = 0; i < 100; i++) { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = min; k < max; k++) + cache.put(k, i); + + tx.commit(); + } + + Map res = checkAndGetAll(false, cache, keys, SCAN, GET); + + for (Integer key : keys) + assertEquals(i, res.get(key)); + + assertEquals(KEYS, res.size()); + } + } + catch (Exception e) { + error("Unexpected error: " + e, e); + + fail("Unexpected error: " + e); + } + } + }, THREADS, "test-thread"); + } + + /** + * @throws Exception If failed. + */ + public void testWaitPreviousTxAck() throws Exception { + testSpi = true; + + startGrid(0); + + client = true; + + final Ignite ignite = startGrid(1); + + final IgniteCache cache = + ignite.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 16)); + + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(1, 1); + cache.put(2, 1); + cache.put(3, 1); + + tx.commit(); + } + + TestRecordingCommunicationSpi clientSpi = TestRecordingCommunicationSpi.spi(ignite); + + clientSpi.blockMessages(new IgniteBiPredicate() { + /** */ + boolean block = true; + + @Override public boolean apply(ClusterNode node, Message msg) { + if (block && msg instanceof MvccAckRequestTx) { + block = false; + + return true; + } + + return false; + } + }); + + IgniteInternalFuture txFut1 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(2, 2); + cache.put(3, 2); + + tx.commit(); + } + + return null; + } + }); + + IgniteInternalFuture txFut2 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(1, 3); + cache.put(2, 3); + + tx.commit(); + } + + // Should see changes mady by both tx1 and tx2. + Map res = checkAndGetAll(false, cache, F.asSet(1, 2, 3), SCAN, GET); + + assertEquals(3, res.get(1)); + assertEquals(3, res.get(2)); + assertEquals(2, res.get(3)); + + return null; + } + }); + + clientSpi.waitForBlocked(); + + Thread.sleep(1000); + + clientSpi.stopBlock(true); + + txFut1.get(); + txFut2.get(); + + Map res = checkAndGetAll(false, cache, F.asSet(1, 2, 3), SCAN, GET); + + assertEquals(3, res.get(1)); + assertEquals(3, res.get(2)); + assertEquals(2, res.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testPartialCommitResultNoVisible() throws Exception { + testSpi = true; + + startGrids(2); + + client = true; + + final Ignite ignite = startGrid(2); + + awaitPartitionMapExchange(); + + final IgniteCache cache = + ignite.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 16)); + + final Integer key1 = primaryKey(ignite(0).cache(cache.getName())); + final Integer key2 = primaryKey(ignite(1).cache(cache.getName())); + + info("Test keys [key1=" + key1 + ", key2=" + key2 + ']'); + + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key1, 1); + cache.put(key2, 1); + + tx.commit(); + } + + Integer val = 1; + + // Allow finish update for key1 and block update for key2. + + TestRecordingCommunicationSpi clientSpi = TestRecordingCommunicationSpi.spi(ignite); + TestRecordingCommunicationSpi srvSpi = TestRecordingCommunicationSpi.spi(ignite(0)); + + for (int i = 0; i < 10; i++) { + info("Iteration: " + i); + + clientSpi.blockMessages(GridNearTxFinishRequest.class, getTestIgniteInstanceName(1)); + + srvSpi.record(GridNearTxFinishResponse.class); + + final Integer newVal = val + 1; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key1, newVal); + cache.put(key2, newVal); + + tx.commit(); + } + + return null; + } + }); + + try { + srvSpi.waitForRecorded(); + + srvSpi.recordedMessages(true); + + assertFalse(fut.isDone()); + + if (i % 2 == 1) { + // Execute one more update to increase counter. + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(primaryKeys(jcache(0), 1, 100_000).get(0), 1); + + tx.commit(); + } + } + + Set keys = new HashSet<>(); + + keys.add(key1); + keys.add(key2); + + Map res; + + res = checkAndGetAll(false, cache, keys, SCAN, GET); + + assertEquals(val, res.get(key1)); + assertEquals(val, res.get(key2)); + + clientSpi.stopBlock(true); + + fut.get(); + + res = checkAndGetAll(false, cache, keys, SCAN, GET); + + assertEquals(newVal, res.get(key1)); + assertEquals(newVal, res.get(key2)); + + val = newVal; + } + finally { + clientSpi.stopBlock(true); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testCleanupWaitsForGet1() throws Exception { + boolean vals[] = {true, false}; + + for (boolean otherPuts : vals) { + for (boolean putOnStart : vals) { + for (boolean inTx : vals) { + cleanupWaitsForGet1(otherPuts, putOnStart, inTx); + + afterTest(); + } + } + } + } + + /** + * @param otherPuts {@code True} to update unrelated keys to increment mvcc counter. + * @param putOnStart {@code True} to put data in cache before getAll. + * @param inTx {@code True} to read inside transaction. + * @throws Exception If failed. + */ + private void cleanupWaitsForGet1(boolean otherPuts, final boolean putOnStart, final boolean inTx) throws Exception { + info("cleanupWaitsForGet [otherPuts=" + otherPuts + + ", putOnStart=" + putOnStart + + ", inTx=" + inTx + "]"); + + testSpi = true; + + client = false; + + final Ignite srv = startGrid(0); + + client = true; + + final Ignite client = startGrid(1); + + awaitPartitionMapExchange(); + + final IgniteCache srvCache = + srv.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 16)); + + final Integer key1 = 1; + final Integer key2 = 2; + + if (putOnStart) { + try (Transaction tx = srv.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + srvCache.put(key1, 0); + srvCache.put(key2, 0); + + tx.commit(); + } + } + + if (otherPuts) { + for (int i = 0; i < 3; i++) { + try (Transaction tx = srv.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + srvCache.put(1_000_000 + i, 99); + + tx.commit(); + } + } + } + + TestRecordingCommunicationSpi clientSpi = TestRecordingCommunicationSpi.spi(client); + + clientSpi.blockMessages(GridNearGetRequest.class, getTestIgniteInstanceName(0)); + + IgniteInternalFuture getFut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + IgniteCache cache = client.cache(srvCache.getName()); + + + Map vals; + + if (inTx) { + try (Transaction tx = client.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) { + vals = checkAndGetAll(false, cache, F.asSet(key1, key2), SCAN, GET); + + tx.rollback(); + } + } + else + vals = checkAndGetAll(false, cache, F.asSet(key1, key2), SCAN, GET); + + if (putOnStart) { + assertEquals(2, vals.size()); + assertEquals(0, (Object)vals.get(key1)); + assertEquals(0, (Object)vals.get(key2)); + } + else + assertEquals(0, vals.size()); + + return null; + } + }, "get-thread"); + + clientSpi.waitForBlocked(); + + for (int i = 0; i < 5; i++) { + try (Transaction tx = srv.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + srvCache.put(key1, i + 1); + srvCache.put(key2, i + 1); + + tx.commit(); + } + } + + clientSpi.stopBlock(true); + + getFut.get(); + + IgniteCache cache = client.cache(srvCache.getName()); + + Map vals = checkAndGetAll(false, cache, F.asSet(key1, key2), SCAN, GET); + + assertEquals(2, vals.size()); + assertEquals(5, (Object)vals.get(key1)); + assertEquals(5, (Object)vals.get(key2)); + } + + /** + * @throws Exception If failed. + */ + public void testCleanupWaitsForGet2() throws Exception { + /* + Simulate case when there are two active transactions modifying the same key + (it is possible if key lock is released but ack message is delayed), and at this moment + query is started. + */ + testSpi = true; + + client = false; + + startGrids(2); + + client = true; + + final Ignite client = startGrid(2); + + awaitPartitionMapExchange(); + + final IgniteCache cache = client.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 16). + setNodeFilter(new TestCacheNodeExcludingFilter(ignite(0).name()))); + + final Integer key1 = 1; + final Integer key2 = 2; + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key1, 0); + cache.put(key2, 0); + + tx.commit(); + } + + TestRecordingCommunicationSpi crdSpi = TestRecordingCommunicationSpi.spi(grid(0)); + + TestRecordingCommunicationSpi clientSpi = TestRecordingCommunicationSpi.spi(client); + + final CountDownLatch getLatch = new CountDownLatch(1); + + clientSpi.closure(new IgniteBiInClosure() { + @Override public void apply(ClusterNode node, Message msg) { + if (msg instanceof MvccAckRequestTx) + doSleep(2000); + } + }); + + crdSpi.closure(new IgniteBiInClosure() { + /** */ + private AtomicInteger cntr = new AtomicInteger(); + + @Override public void apply(ClusterNode node, Message msg) { + if (msg instanceof MvccSnapshotResponse) { + if (cntr.incrementAndGet() == 2) { + getLatch.countDown(); + + doSleep(1000); + } + } + } + }); + + final IgniteInternalFuture putFut1 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key1, 1); + + tx.commit(); + } + + return null; + } + }, "put1"); + + final IgniteInternalFuture putFut2 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key1, 2); + + tx.commit(); + } + + return null; + } + }, "put2"); + + IgniteInternalFuture getFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + U.await(getLatch); + + while (!putFut1.isDone() || !putFut2.isDone()) { + Map vals1 = checkAndGetAll(false, cache, F.asSet(key1, key2), SCAN); + Map vals2 = checkAndGetAll(false, cache, F.asSet(key1, key2), GET); + + assertEquals(2, vals1.size()); + assertEquals(2, vals2.size()); + } + + return null; + } + }, 4, "get-thread"); + + putFut1.get(); + putFut2.get(); + getFut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testCleanupWaitsForGet3() throws Exception { + for (int i = 0; i < 4; i++) { + cleanupWaitsForGet3(i + 1); + + afterTest(); + } + } + + /** + * @param updates Number of updates. + * @throws Exception If failed. + */ + private void cleanupWaitsForGet3(int updates) throws Exception { + /* + Simulate case when coordinator assigned query version has active transaction, + query is delayed, after this active transaction finish and the same key is + updated several more times before query starts. + */ + testSpi = true; + + client = false; + + startGrids(1); + + client = true; + + final Ignite client = startGrid(1); + + awaitPartitionMapExchange(); + + final IgniteCache cache = client.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 16)); + + final Integer key1 = 1; + final Integer key2 = 2; + + for (int i = 0; i < updates; i++) { + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key1, i); + cache.put(key2, i); + + tx.commit(); + } + } + + TestRecordingCommunicationSpi crdSpi = TestRecordingCommunicationSpi.spi(grid(0)); + + TestRecordingCommunicationSpi clientSpi = TestRecordingCommunicationSpi.spi(client); + + clientSpi.blockMessages(new IgniteBiPredicate() { + /** */ + private boolean blocked; + + @Override public boolean apply(ClusterNode node, Message msg) { + if (!blocked && (msg instanceof MvccAckRequestTx)) { + blocked = true; + + return true; + } + return false; + } + }); + + final IgniteInternalFuture putFut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key2, 3); + + tx.commit(); + } + + return null; + } + }, "put"); + + clientSpi.waitForBlocked(); + + for (int i = 0; i < updates; i++) { + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key1, i + 3); + + tx.commit(); + } + } + + // Delay version for getAll. + crdSpi.blockMessages(new IgniteBiPredicate() { + /** */ + private boolean blocked; + + @Override public boolean apply(ClusterNode node, Message msg) { + if (!blocked && (msg instanceof MvccSnapshotResponse)) { + blocked = true; + + return true; + } + return false; + } + }); + + final IgniteInternalFuture getFut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + final Map res1 = checkAndGetAll(false, cache, F.asSet(key1, key2), SCAN); + final Map res2 = checkAndGetAll(false, cache, F.asSet(key1, key2), GET); + + assertEquals(2, res1.size()); + assertEquals(2, res2.size()); + + return null; + } + }, "get"); + + crdSpi.waitForBlocked(); + + clientSpi.stopBlock(true); + + putFut.get(); + + for (int i = 0; i < updates; i++) { + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key2, i + 4); + + tx.commit(); + } + } + + crdSpi.stopBlock(true); + + getFut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_SingleNode_GetAll() throws Exception { + putAllGetAll(null, 1, 0, 0, 64, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_SingleNode_SinglePartition_GetAll() throws Exception { + putAllGetAll(null, 1, 0, 0, 1, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups0_GetAll() throws Exception { + putAllGetAll(null, 4, 2, 0, 64, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups0_Persistence_GetAll() throws Exception { + persistence = true; + + testPutAllGetAll_ClientServer_Backups0_GetAll(); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_GetAll() throws Exception { + putAllGetAll(null, 4, 2, 1, 64, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups2_GetAll() throws Exception { + putAllGetAll(null, 4, 2, 2, 64, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_RestartCoordinator_GetAll() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD, 4, 2, 1, 64, null, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_SingleNode_Scan() throws Exception { + putAllGetAll(null, 1, 0, 0, 64, null, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_SingleNode_SinglePartition_Scan() throws Exception { + putAllGetAll(null, 1, 0, 0, 1, null, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups0_Scan() throws Exception { + putAllGetAll(null, 4, 2, 0, 64, null, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups0_Persistence_Scan() throws Exception { + persistence = true; + + testPutAllGetAll_ClientServer_Backups0_Scan(); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_Scan() throws Exception { + putAllGetAll(null, 4, 2, 1, 64, null, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups2_Scan() throws Exception { + putAllGetAll(null, 4, 2, 2, 64, null, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_RestartCoordinator_Scan() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD, 4, 2, 1, 64, null, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_Restart_Scan() throws Exception { + putAllGetAll(RestartMode.RESTART_RND_SRV, 4, 2, 1, 64, null, SCAN, PUT); + } + + + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, null, false, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_WithRemoves_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, null, true, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, null, false, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_WithRemoves_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, null, true, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_ClientServer_Backups0() throws Exception { + accountsTxReadAll(4, 2, 0, 64, null, false, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_WithRemoves_ClientServer_Backups0() throws Exception { + accountsTxReadAll(4, 2, 0, 64, null, true, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_ClientServer_Backups1() throws Exception { + accountsTxReadAll(4, 2, 1, 64, null, false, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_WithRemoves_ClientServer_Backups1() throws Exception { + accountsTxReadAll(4, 2, 1, 64, null, true, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, null, false, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxGetAll_WithRemoves_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, null, true, GET, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, null, false, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_WithRemoves_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, null, true, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, null, false, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_WithRemoves_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, null, true, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_ClientServer_Backups0() throws Exception { + accountsTxReadAll(4, 2, 0, 64, null, false, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_WithRemoves_ClientServer_Backups0() throws Exception { + accountsTxReadAll(4, 2, 0, 64, null, true, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_ClientServer_Backups1() throws Exception { + accountsTxReadAll(4, 2, 1, 64, null, false, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_WithRemoves_ClientServer_Backups1() throws Exception { + accountsTxReadAll(4, 2, 1, 64, null, true, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, null, false, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxScan_WithRemoves_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, null, true, SCAN, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticTxGetAllReadsSnapshot_SingleNode_SinglePartition() throws Exception { + txReadsSnapshot(1, 0, 0, 1, true, GET); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticTxGetAllReadsSnapshot_ClientServer() throws Exception { + txReadsSnapshot(4, 2, 1, 64, true, GET); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticTxGetAllReadsSnapshot_SingleNode() throws Exception { + txReadsSnapshot(1, 0, 0, 64, false, GET); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticTxGetAllReadsSnapshot_SingleNode_SinglePartition() throws Exception { + txReadsSnapshot(1, 0, 0, 1, false, GET); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticTxGetAllReadsSnapshot_ClientServer() throws Exception { + txReadsSnapshot(4, 2, 1, 64, false, GET); + } + +// TODO: IGNITE-7371 +// /** +// * @throws Exception If failed. +// */ +// public void testPessimisticTxScanReadsSnapshot_SingleNode_SinglePartition() throws Exception { +// txReadsSnapshot(1, 0, 0, 1, true, SCAN); +// } +// +// /** +// * @throws Exception If failed. +// */ +// public void testPessimisticTxScanReadsSnapshot_ClientServer() throws Exception { +// txReadsSnapshot(4, 2, 1, 64, true, SCAN); +// } +// +// /** +// * @throws Exception If failed. +// */ +// public void testOptimisticTxScanReadsSnapshot_SingleNode() throws Exception { +// txReadsSnapshot(1, 0, 0, 64, false, SCAN); +// } +// +// /** +// * @throws Exception If failed. +// */ +// public void testOptimisticTxScanReadsSnapshot_SingleNode_SinglePartition() throws Exception { +// txReadsSnapshot(1, 0, 0, 1, false, SCAN); +// } +// +// /** +// * @throws Exception If failed. +// */ +// public void testOptimisticTxScanReadsSnapshot_ClientServer() throws Exception { +// txReadsSnapshot(4, 2, 1, 64, false, SCAN); +// } + + /** + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param pessimistic If {@code true} uses pessimistic tx, otherwise optimistic. + * @param readMode Read mode. + * @throws Exception If failed. + */ + private void txReadsSnapshot( + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + final boolean pessimistic, + ReadMode readMode + ) throws Exception { + final int ACCOUNTS = 20; + + final int ACCOUNT_START_VAL = 1000; + + final int writers = 4; + + final int readers = 4; + + final TransactionConcurrency concurrency; + final TransactionIsolation isolation; + + if (pessimistic) { + concurrency = PESSIMISTIC; + isolation = REPEATABLE_READ; + } + else { + concurrency = OPTIMISTIC; + isolation = SERIALIZABLE; + } + + final IgniteInClosure> init = new IgniteInClosure>() { + @Override public void apply(IgniteCache cache) { + final IgniteTransactions txs = cache.unwrap(Ignite.class).transactions(); + + Map accounts = new HashMap<>(); + + for (int i = 0; i < ACCOUNTS; i++) + accounts.put(i, new MvccTestAccount(ACCOUNT_START_VAL, 1)); + + try (Transaction tx = txs.txStart(concurrency, isolation)) { + cache.putAll(accounts); + + tx.commit(); + } + } + }; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + cnt++; + + Integer id1 = rnd.nextInt(ACCOUNTS); + Integer id2 = rnd.nextInt(ACCOUNTS); + + while (id1.equals(id2)) + id2 = rnd.nextInt(ACCOUNTS); + + TreeSet keys = new TreeSet<>(); + + keys.add(id1); + keys.add(id2); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + MvccTestAccount a1; + MvccTestAccount a2; + + Map accounts = checkAndGetAll(false, cache.cache, keys, readMode); + + a1 = accounts.get(id1); + a2 = accounts.get(id2); + + assertNotNull(a1); + assertNotNull(a2); + + cache.cache.put(id1, new MvccTestAccount(a1.val + 1, 1)); + cache.cache.put(id2, new MvccTestAccount(a2.val - 1, 1)); + + tx.commit(); + } + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + Map accounts = new HashMap<>(); + + if (pessimistic) { + try (Transaction tx = txs.txStart(concurrency, isolation)) { + int remaining = ACCOUNTS; + + do { + int readCnt = rnd.nextInt(remaining) + 1; + + Set readKeys = new TreeSet<>(); + + for (int i = 0; i < readCnt; i++) + readKeys.add(accounts.size() + i); + + Map readRes = + checkAndGetAll(false, cache.cache, readKeys, readMode); + + assertEquals(readCnt, readRes.size()); + + accounts.putAll(readRes); + + remaining = ACCOUNTS - accounts.size(); + } + while (remaining > 0); + + validateSum(accounts); + + tx.commit(); + + cnt++; + } + finally { + cache.readUnlock(); + } + } + else { + try (Transaction tx = txs.txStart(concurrency, isolation)) { + int remaining = ACCOUNTS; + + do { + int readCnt = rnd.nextInt(remaining) + 1; + + if (rnd.nextInt(3) == 0) { + for (int i = 0; i < readCnt; i++) { + Integer key = rnd.nextInt(ACCOUNTS); + + MvccTestAccount account = + (MvccTestAccount)checkAndGet(false, cache.cache, key, readMode); + + assertNotNull(account); + + accounts.put(key, account); + } + } + else { + Set readKeys = new LinkedHashSet<>(); + + for (int i = 0; i < readCnt; i++) + readKeys.add(rnd.nextInt(ACCOUNTS)); + + Map readRes = + checkAndGetAll(false, cache.cache, readKeys, readMode); + + assertEquals(readKeys.size(), readRes.size()); + + accounts.putAll(readRes); + } + + remaining = ACCOUNTS - accounts.size(); + } + while (remaining > 0); + + validateSum(accounts); + + cnt++; + + tx.commit(); + } + catch (TransactionOptimisticException ignore) { + // No-op. + } + finally { + cache.readUnlock(); + } + } + } + + info("Reader finished, txs: " + cnt); + } + + /** + * @param accounts Read accounts. + */ + private void validateSum(Map accounts) { + int sum = 0; + + for (int i = 0; i < ACCOUNTS; i++) { + MvccTestAccount account = accounts.get(i); + + assertNotNull(account); + + sum += account.val; + } + + assertEquals(ACCOUNTS * ACCOUNT_START_VAL, sum); + } + }; + + readWriteTest( + null, + srvs, + clients, + cacheBackups, + cacheParts, + writers, + readers, + DFLT_TEST_TIME, + null, + init, + writer, + reader); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceScanConsistency_SingleNode_SinglePartition() throws Exception { + operationsSequenceConsistency(1, 0, 0, 1, SCAN); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceScanConsistency_SingleNode() throws Exception { + operationsSequenceConsistency(1, 0, 0, 64, SCAN); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceScanConsistency_ClientServer_Backups0() throws Exception { + operationsSequenceConsistency(4, 2, 0, 64, SCAN); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceScanConsistency_ClientServer_Backups1() throws Exception { + operationsSequenceConsistency(4, 2, 1, 64, SCAN); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceGetConsistency_SingleNode_SinglePartition() throws Exception { + operationsSequenceConsistency(1, 0, 0, 1, GET); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceGetConsistency_SingleNode() throws Exception { + operationsSequenceConsistency(1, 0, 0, 64, GET); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceGetConsistency_ClientServer_Backups0() throws Exception { + operationsSequenceConsistency(4, 2, 0, 64, GET); + } + + /** + * @throws Exception If failed + */ + public void testOperationsSequenceGetConsistency_ClientServer_Backups1() throws Exception { + operationsSequenceConsistency(4, 2, 1, 64, GET); + } + + /** + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param readMode Read mode. + * @throws Exception If failed. + */ + private void operationsSequenceConsistency( + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + ReadMode readMode + ) + throws Exception + { + final int writers = 4; + + final int readers = 4; + + final long time = 10_000; + + final AtomicInteger keyCntr = new AtomicInteger(); + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + Integer key = keyCntr.incrementAndGet(); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.cache.put(key, new Value(idx, cnt++)); + + tx.commit(); + } + + if (key > 100_000) + break; + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Set keys = new HashSet(); + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + Map> vals = new HashMap<>(); + + switch (readMode) { + case SCAN: + for (Cache.Entry e : cache.cache) { + Value val = e.getValue(); + + assertNotNull(val); + + TreeSet cntrs = vals.get(val.key); + + if (cntrs == null) + vals.put(val.key, cntrs = new TreeSet<>()); + + boolean add = cntrs.add(val.cnt); + + assertTrue(add); + } + + break; + + case GET: + for (int i = keys.size(); i < keyCntr.get(); i++) + keys.add(i); + + Iterable> entries = cache.cache.getAll(keys).entrySet(); + + for (Map.Entry e : entries) { + Value val = e.getValue(); + + assertNotNull(val); + + TreeSet cntrs = vals.get(val.key); + + if (cntrs == null) + vals.put(val.key, cntrs = new TreeSet<>()); + + boolean add = cntrs.add(val.cnt); + + assertTrue(add); + } + + break; + + default: + fail("Unsupported read mode: " + readMode.name() + '.'); + } + + for (TreeSet readCntrs : vals.values()) { + for (int i = 0; i < readCntrs.size(); i++) + assertTrue(readCntrs.contains(i)); + } + } + finally { + cache.readUnlock(); + } + } + } + }; + + readWriteTest( + null, + srvs, + clients, + cacheBackups, + cacheParts, + writers, + readers, + time, + null, + null, + writer, + reader); + } + + /** + * TODO IGNITE-5935 enable when recovery is implemented. + * + * @throws Exception If failed. + */ + public void _testNodesRestartNoHang() throws Exception { + final int srvs = 4; + final int clients = 4; + final int writers = 6; + final int readers = 2; + + final int KEYS = 100_000; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Map map = new TreeMap<>(); + + int cnt = 0; + + while (!stop.get()) { + int keys = rnd.nextInt(32) + 1; + + while (map.size() < keys) + map.put(rnd.nextInt(KEYS), cnt); + + TestCache cache = randomCache(caches, rnd); + + try { + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + TransactionConcurrency concurrency; + TransactionIsolation isolation; + + switch (rnd.nextInt(3)) { + case 0: { + concurrency = PESSIMISTIC; + isolation = REPEATABLE_READ; + + break; + } + case 1: { + concurrency = OPTIMISTIC; + isolation = REPEATABLE_READ; + + break; + } + case 2: { + concurrency = OPTIMISTIC; + isolation = SERIALIZABLE; + + break; + } + default: { + fail(); + + return; + } + } + + try (Transaction tx = txs.txStart(concurrency, isolation)) { + if (rnd.nextBoolean()) { + Map res = checkAndGetAll(false, cache.cache, map.keySet(), + rnd.nextBoolean() ? GET : SCAN); + + assertNotNull(res); + } + + cache.cache.putAll(map); + + tx.commit(); + } + catch (TransactionOptimisticException e) { + assertEquals(SERIALIZABLE, isolation); + } + catch (Exception e) { + Assert.assertTrue("Unexpected error: " + e, X.hasCause(e, ClusterTopologyException.class)); + } + } + finally { + cache.readUnlock(); + } + + map.clear(); + + cnt++; + } + + info("Writer done, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + Set keys = new LinkedHashSet<>(); + + while (!stop.get()) { + int keyCnt = rnd.nextInt(64) + 1; + + while (keys.size() < keyCnt) + keys.add(rnd.nextInt(KEYS)); + + TestCache cache = randomCache(caches, rnd); + + Map map; + + try { + map = checkAndGetAll(false, cache.cache, keys, rnd.nextBoolean() ? GET : SCAN); + + assertNotNull(map); + } + finally { + cache.readUnlock(); + } + + keys.clear(); + } + } + }; + + readWriteTest( + RestartMode.RESTART_RND_SRV, + srvs, + clients, + 1, + 256, + writers, + readers, + DFLT_TEST_TIME, + null, + null, + writer, + reader); + + for (Ignite node : G.allGrids()) + checkActiveQueriesCleanup(node); + } + + /** + * @throws Exception If failed. + */ + public void testActiveQueryCleanupOnNodeFailure() throws Exception { + testSpi = true; + + final Ignite srv = startGrid(0); + + srv.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1024)); + + client = true; + + final Ignite client = startGrid(1); + + TestRecordingCommunicationSpi srvSpi = TestRecordingCommunicationSpi.spi(srv); + + srvSpi.blockMessages(GridNearGetResponse.class, getTestIgniteInstanceName(1)); + + TestRecordingCommunicationSpi.spi(client).blockMessages(MvccAckRequestQueryCntr.class, + getTestIgniteInstanceName(0)); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); + + cache.getAll(F.asSet(1, 2, 3)); + + return null; + } + }); + + srvSpi.waitForBlocked(); + + assertFalse(fut.isDone()); + + stopGrid(1); + + checkActiveQueriesCleanup(ignite(0)); + + verifyCoordinatorInternalState(); + + try { + fut.get(); + } + catch (Exception ignore) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testRebalanceSimple() throws Exception { + Ignite srv0 = startGrid(0); + + IgniteCache cache = (IgniteCache)srv0.createCache( + cacheConfiguration(PARTITIONED, FULL_SYNC, 0, DFLT_PARTITION_COUNT)); + + Map map; + Map resMap; + + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + map = new HashMap<>(); + + for (int i = 0; i < DFLT_PARTITION_COUNT * 3; i++) + map.put(i, i); + + cache.putAll(map); + + tx.commit(); + } + + startGrid(1); + + awaitPartitionMapExchange(); + + resMap = checkAndGetAll(false, cache, map.keySet(), GET, SCAN); + + assertEquals(map.size(), resMap.size()); + + for (int i = 0; i < map.size(); i++) + assertEquals(i, (Object)resMap.get(i)); + + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int i = 0; i < DFLT_PARTITION_COUNT * 3; i++) + map.put(i, i + 1); + + cache.putAll(map); + + tx.commit(); + } + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int i = 0; i < DFLT_PARTITION_COUNT * 3; i++) + map.put(i, i + 2); + + cache.putAll(map); + + tx.commit(); + } + + startGrid(2); + + awaitPartitionMapExchange(); + + resMap = checkAndGetAll(false, cache, map.keySet(), GET, SCAN); + + assertEquals(map.size(), map.size()); + + for (int i = 0; i < map.size(); i++) + assertEquals(i + 2, (Object)resMap.get(i)); + + // Run fake transaction + try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + Integer val = cache.get(0); + + cache.put(0, val); + + tx.commit(); + } + + resMap = checkAndGetAll(false, cache, map.keySet(), GET, SCAN); + + assertEquals(map.size(), map.size()); + + for (int i = 0; i < map.size(); i++) + assertEquals(i + 2, (Object)resMap.get(i)); + } + + /** + * @throws Exception If failed. + */ + public void testRebalanceWithRemovedValuesSimple() throws Exception { + Ignite node = startGrid(0); + + IgniteTransactions txs = node.transactions(); + + final IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, 64)); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = 0; k < 100; k++) + cache.remove(k); + + tx.commit(); + } + + Map expVals = new HashMap<>(); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = 100; k < 200; k++) { + cache.put(k, k); + + expVals.put(k, k); + } + + tx.commit(); + } + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int k = 100; k < 200; k++) { + if (k % 2 == 0) { + cache.remove(k); + + expVals.remove(k); + } + } + + tx.commit(); + } + + startGrid(1); + + awaitPartitionMapExchange(); + + checkValues(expVals, jcache(1)); + + stopGrid(0); + + checkValues(expVals, jcache(1)); + } + + /** + * @throws Exception If failed. + */ + public void testTxPrepareFailureSimplePessimisticTx() throws Exception { + txPrepareFailureSimple(PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testTxPrepareFailureSimpleSerializableTx() throws Exception { + txPrepareFailureSimple(OPTIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testTxPrepareFailureSimpleOptimisticTx() throws Exception { + txPrepareFailureSimple(OPTIMISTIC, REPEATABLE_READ); + } + + /** + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @throws Exception If failed. + */ + private void txPrepareFailureSimple( + final TransactionConcurrency concurrency, + final TransactionIsolation isolation + ) throws Exception { + testSpi = true; + + startGrids(3); + + client = true; + + final Ignite client = startGrid(3); + + final IgniteCache cache = client.createCache( + cacheConfiguration(PARTITIONED, FULL_SYNC, 0, DFLT_PARTITION_COUNT)); + + final Integer key1 = primaryKey(jcache(1)); + final Integer key2 = primaryKey(jcache(2)); + + TestRecordingCommunicationSpi srv1Spi = TestRecordingCommunicationSpi.spi(ignite(1)); + + srv1Spi.blockMessages(GridNearTxPrepareResponse.class, client.name()); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + try { + try (Transaction tx = client.transactions().txStart(concurrency, isolation)) { + cache.put(key1, 1); + cache.put(key2, 2); + + tx.commit(); + } + + fail(); + } + catch (ClusterTopologyException e) { + info("Expected exception: " + e); + + assertNotNull(e.retryReadyFuture()); + + e.retryReadyFuture().get(); + } + + return null; + } + }, "tx-thread"); + + srv1Spi.waitForBlocked(); + + assertFalse(fut.isDone()); + + stopGrid(1); + + fut.get(); + + assertNull(cache.get(key1)); + assertNull(cache.get(key2)); + + try (Transaction tx = client.transactions().txStart(concurrency, isolation)) { + cache.put(key1, 1); + cache.put(key2, 2); + + tx.commit(); + } + + assertEquals(1, checkAndGet(false, cache, key1, GET, SCAN)); + assertEquals(2, checkAndGet(false, cache, key2, GET, SCAN)); + } + + /** + * @throws Exception If failed. + */ + public void testSerializableTxRemap() throws Exception { + testSpi = true; + + startGrids(2); + + client = true; + + final Ignite client = startGrid(2); + + final IgniteCache cache = client.createCache( + cacheConfiguration(PARTITIONED, FULL_SYNC, 0, DFLT_PARTITION_COUNT)); + + final Map vals = new HashMap<>(); + + for (int i = 0; i < 100; i++) + vals.put(i, i); + + TestRecordingCommunicationSpi clientSpi = TestRecordingCommunicationSpi.spi(ignite(2)); + + clientSpi.blockMessages(new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode node, Message msg) { + return msg instanceof GridNearTxPrepareRequest; + } + }); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = client.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) { + cache.putAll(vals); + + tx.commit(); + } + + return null; + } + }, "tx-thread"); + + clientSpi.waitForBlocked(2); + + this.client = false; + + startGrid(3); + + assertFalse(fut.isDone()); + + clientSpi.stopBlock(); + + fut.get(); + + for (Ignite node : G.allGrids()) + checkValues(vals, node.cache(cache.getName())); + } + + + /** + * @throws Exception If failed. + */ + public void testMvccCoordinatorChangeSimple() throws Exception { + Ignite srv0 = startGrid(0); + + final List cacheNames = new ArrayList<>(); + + for (CacheConfiguration ccfg : cacheConfigurations()) { + ccfg.setName("cache-" + cacheNames.size()); + + cacheNames.add(ccfg.getName()); + + srv0.createCache(ccfg); + } + + checkPutGet(cacheNames); + + for (int i = 0; i < 3; i++) { + startGrid(i + 1); + + checkPutGet(cacheNames); + + checkCoordinatorsConsistency(null); + } + + client = true; + + for (int i = 0; i < 3; i++) { + Ignite node = startGrid(i + 4); + + // Init client caches outside of transactions. + for (String cacheName : cacheNames) + node.cache(cacheName); + + checkPutGet(cacheNames); + + checkCoordinatorsConsistency(null); + } + + for (int i = 0; i < 3; i++) { + stopGrid(i); + + awaitPartitionMapExchange(); + + checkPutGet(cacheNames); + + checkCoordinatorsConsistency(null); + } + } + + /** + * @param cacheNames Cache names. + */ + private void checkPutGet(List cacheNames) { + List nodes = G.allGrids(); + + assertFalse(nodes.isEmpty()); + + Ignite putNode = nodes.get(ThreadLocalRandom.current().nextInt(nodes.size())); + + Map vals = new HashMap(); + + Integer val = ThreadLocalRandom.current().nextInt(); + + for (int i = 0; i < 10; i++) + vals.put(i, val); + + TransactionConcurrency concurrency; + TransactionIsolation isolation; + + if (ThreadLocalRandom.current().nextBoolean()) { + concurrency = PESSIMISTIC; + isolation = REPEATABLE_READ; + } + else { + concurrency = OPTIMISTIC; + isolation = SERIALIZABLE; + } + + try (Transaction tx = putNode.transactions().txStart(concurrency, isolation)) { + for (String cacheName : cacheNames) + putNode.cache(cacheName).putAll(vals); + + tx.commit(); + } + + for (Ignite node : nodes) { + for (String cacheName : cacheNames) { + Map res = checkAndGetAll(false, node.cache(cacheName), vals.keySet(), SCAN, GET); + + assertEquals(vals, res); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testMvccCoordinatorInfoConsistency() throws Exception { + for (int i = 0; i < 4; i++) { + startGrid(i); + + if (persistence && i == 0) + ignite(i).active(true); + + checkCoordinatorsConsistency(i + 1); + } + + client = true; + + startGrid(4); + + checkCoordinatorsConsistency(5); + + startGrid(5); + + checkCoordinatorsConsistency(6); + + client = false; + + stopGrid(0); + + awaitPartitionMapExchange(); + + checkCoordinatorsConsistency(5); + } + + /** + * @throws Exception If failed. + */ + public void testMvccCoordinatorInfoConsistency_Persistence() throws Exception { + persistence = true; + + testMvccCoordinatorInfoConsistency(); + } + + /** + * @param expNodes Expected nodes number. + */ + private void checkCoordinatorsConsistency(@Nullable Integer expNodes) { + List nodes = G.allGrids(); + + if (expNodes != null) + assertEquals(expNodes, (Integer)nodes.size()); + + MvccCoordinator crd = null; + + for (Ignite node : G.allGrids()) { + MvccCoordinator crd0 = mvccProcessor(node).currentCoordinator(); + + if (crd != null) + assertEquals(crd, crd0); + else + crd = crd0; + } + } + + /** + * @throws Exception If failed. + */ + public void testGetVersionRequestFailover() throws Exception { + final int NODES = 5; + + testSpi = true; + + startGridsMultiThreaded(NODES - 1); + + client = true; + + Ignite client = startGrid(NODES - 1); + + final List cacheNames = new ArrayList<>(); + + final Map vals = new HashMap<>(); + + for (int i = 0; i < 100; i++) + vals.put(i, i); + + for (CacheConfiguration ccfg : cacheConfigurations()) { + ccfg.setName("cache-" + cacheNames.size()); + + ccfg.setNodeFilter(new TestCacheNodeExcludingFilter(getTestIgniteInstanceName(0))); + + cacheNames.add(ccfg.getName()); + + IgniteCache cache = client.createCache(ccfg); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + writeAllByMode(cache, vals, PUT, INTEGER_CODEC); + + tx.commit(); + } + } + + final AtomicInteger nodeIdx = new AtomicInteger(1); + + final AtomicBoolean done = new AtomicBoolean(); + + try { + IgniteInternalFuture getFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + Ignite node = ignite(nodeIdx.getAndIncrement()); + + int cnt = 0; + + while (!done.get()) { + for (String cacheName : cacheNames) { + // TODO IGNITE-6754 add SQL and SCAN support. + Map res = readAllByMode(node.cache(cacheName), vals.keySet(), GET, INTEGER_CODEC); + + assertEquals(vals, res); + } + + cnt++; + } + + log.info("Finished [node=" + node.name() + ", cnt=" + cnt + ']'); + + return null; + } + }, NODES - 1, "get-thread"); + + doSleep(1000); + + TestRecordingCommunicationSpi crdSpi = TestRecordingCommunicationSpi.spi(ignite(0)); + + crdSpi.blockMessages(new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode node, Message msg) { + return msg instanceof MvccSnapshotResponse; + } + }); + + crdSpi.waitForBlocked(); + + stopGrid(0); + + doSleep(1000); + + done.set(true); + + getFut.get(); + } + finally { + done.set(true); + } + } + + /** + * @throws Exception If failed. + */ + public void testLoadWithStreamer() throws Exception { + startGridsMultiThreaded(5); + + client = true; + + startGrid(5); + + Ignite node = ignite(0); + + IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 2, 64)); + + final int KEYS = 10_000; + + Map data = new HashMap<>(); + + try (IgniteDataStreamer streamer = node.dataStreamer(cache.getName())) { + for (int i = 0; i < KEYS; i++) { + streamer.addData(i, i); + + data.put(i, i); + } + } + + checkValues(data, cache); + + checkCacheData(data, cache.getName()); + + checkPutGet(F.asList(cache.getName())); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_SingleNode_SinglePartition_Get() throws Exception { + int[] nValues = {3, 5, 10}; + + for (int n : nValues) { + updateNObjectsTest(n, 1, 0, 0, 1, 10_000, null, GET, PUT, null); + + afterTest(); + } + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_SingleNode_Get() throws Exception { + int[] nValues = {3, 5, 10}; + + for (int n : nValues) { + updateNObjectsTest(n, 1, 0, 0, 64, 10_000, null, GET, PUT, null); + + afterTest(); + } + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_SingleNode_SinglePartition_Scan() throws Exception { + int[] nValues = {3, 5, 10}; + + for (int n : nValues) { + updateNObjectsTest(n, 1, 0, 0, 1, 10_000, null, SCAN, PUT, null); + + afterTest(); + } + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_SingleNode_Scan() throws Exception { + int[] nValues = {3, 5, 10}; + + for (int n : nValues) { + updateNObjectsTest(n, 1, 0, 0, 64, 10_000, null, SCAN, PUT, null); + + afterTest(); + } + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_ClientServer_Backups2_Get() throws Exception { + int[] nValues = {3, 5, 10}; + + for (int n : nValues) { + updateNObjectsTest(n, 4, 2, 2, DFLT_PARTITION_COUNT, 10_000, null, GET, PUT, null); + + afterTest(); + } + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_ClientServer_Backups1_Scan() throws Exception { + int[] nValues = {3, 5, 10}; + + for (int n : nValues) { + updateNObjectsTest(n, 2, 1, 1, DFLT_PARTITION_COUNT, 10_000, null, SCAN, PUT, null); + + afterTest(); + } + } + + /** + * @throws Exception If failed. + */ + public void testImplicitPartsScan_SingleNode_SinglePartition() throws Exception { + doImplicitPartsScanTest(1, 0, 0, 1, 10_000); + } + + + /** + * @throws Exception If failed. + */ + public void testImplicitPartsScan_SingleNode() throws Exception { + doImplicitPartsScanTest(1, 0, 0, 64, 10_000); + } + + /** + * @throws Exception If failed. + */ + public void testImplicitPartsScan_ClientServer_Backups0() throws Exception { + doImplicitPartsScanTest(4, 2, 0, 64, 10_000); + } + + /** + * @throws Exception If failed. + */ + public void testImplicitPartsScan_ClientServer_Backups1() throws Exception { + doImplicitPartsScanTest(4, 2, 1, 64, 10_000); + } + + /** + * @throws Exception If failed. + */ + public void testImplicitPartsScan_ClientServer_Backups2() throws Exception { + doImplicitPartsScanTest(4, 2, 2, 64, 10_000); + } + + /** + * @param srvs Number of server nodes. + * @param clients Number of client nodes. + * @param cacheBackups Number of cache backups. + * @param cacheParts Number of cache partitions. + * @param time Test time. + * @throws Exception If failed. + */ + private void doImplicitPartsScanTest( + final int srvs, + final int clients, + int cacheBackups, + int cacheParts, + long time) throws Exception { + final int KEYS_PER_PART = 20; + + final int writers = 4; + + final int readers = 4; + + Map> keysByParts = new HashMap<>(); + + final IgniteInClosure> init = new IgniteInClosure>() { + @Override public void apply(IgniteCache cache) { + final IgniteTransactions txs = cache.unwrap(Ignite.class).transactions(); + + for (int i = 0; i < cacheParts; i++) { + List keys = new ArrayList<>(); + + keysByParts.put(i, keys); + } + + Affinity aff = affinity(cache); + + int cntr = 0; + int key = 0; + + while (cntr < KEYS_PER_PART * cacheParts) { + int part = aff.partition(key); + + List keys = keysByParts.get(part); + + if (keys.size() < KEYS_PER_PART) { + keys.add(key); + + cntr++; + } + + key++; + } + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (List keys : keysByParts.values()) + for (Integer k : keys) + cache.put(k, new MvccTestAccount(0, 1)); + + tx.commit(); + } + } + }; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!stop.get()) { + int part = rnd.nextInt(cacheParts); + + List partKeys = keysByParts.get(part); + + TestCache cache = randomCache(caches, rnd); + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + Integer k1 = partKeys.get(rnd.nextInt(KEYS_PER_PART)); + Integer k2 = partKeys.get(rnd.nextInt(KEYS_PER_PART)); + + while (k1.equals(k2)) + k2 = partKeys.get(rnd.nextInt(KEYS_PER_PART)); + + TreeSet keys = new TreeSet<>(); + + keys.add(k1); + keys.add(k2); + + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + Map accs = cache.cache.getAll(keys); + + MvccTestAccount acc1 = accs.get(k1); + MvccTestAccount acc2 = accs.get(k2); + + assertNotNull(acc1); + assertNotNull(acc2); + + cache.cache.put(k1, new MvccTestAccount(acc1.val + 1, acc1.updateCnt + 1)); + cache.cache.put(k2, new MvccTestAccount(acc2.val - 1, acc2.updateCnt + 1)); + + tx.commit(); + } + finally { + cache.readUnlock(); + } + } + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!stop.get()) { + int part = rnd.nextInt(cacheParts); + + TestCache cache = randomCache(caches, rnd); + + try { + Affinity aff = affinity(cache.cache); + + ScanQuery qry = new ScanQuery<>(part); + + List> res = cache.cache.query(qry).getAll(); + + int sum = 0; + + for (Cache.Entry entry : res) { + Integer key = entry.getKey(); + MvccTestAccount acc = entry.getValue(); + + assertEquals(part, aff.partition(key)); + + sum += acc.val; + } + + assertEquals(0, sum); + + } + finally { + cache.readUnlock(); + } + + if (idx == 0) { + cache = randomCache(caches, rnd); + + try { + ScanQuery qry = new ScanQuery<>(); + + List> res = cache.cache.query(qry).getAll(); + + int sum = 0; + + for (Cache.Entry entry : res) { + Integer key = entry.getKey(); + MvccTestAccount acc = entry.getValue(); + + sum += acc.val; + } + + assertEquals(0, sum); + } + finally { + cache.readUnlock(); + } + } + } + } + }; + + readWriteTest( + null, + srvs, + clients, + cacheBackups, + cacheParts, + writers, + readers, + time, + null, + init, + writer, + reader); + } + + /** + * @throws IgniteCheckedException If failed. + */ + public void testSize() throws Exception { + Ignite node = startGrid(0); + + IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1)); + + assertEquals(cache.size(), 0); + + final int KEYS = 10; + + for (int i = 0; i < KEYS; i++) { + final Integer key = i; + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key, i); + + tx.commit(); + } + + assertEquals(i + 1, cache.size()); + } + + for (int i = 0; i < KEYS; i++) { + final Integer key = i; + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key, i); + + tx.commit(); + } + + assertEquals(KEYS, cache.size()); + } + + int size = KEYS; + + for (int i = 0; i < KEYS; i++) { + if (i % 2 == 0) { + final Integer key = i; + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.remove(key); + + tx.commit(); + } + + size--; + + assertEquals(size, cache.size()); + } + } + + // Check size does not change if remove already removed keys. + for (int i = 0; i < KEYS; i++) { + if (i % 2 == 0) { + final Integer key = i; + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.remove(key); + + tx.commit(); + } + + assertEquals(size, cache.size()); + } + } + + for (int i = 0; i < KEYS; i++) { + if (i % 2 == 0) { + final Integer key = i; + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key, i); + + tx.commit(); + } + + size++; + + assertEquals(size, cache.size()); + } + } + } + + /** + * @throws IgniteCheckedException If failed. + */ + public void testInternalApi() throws Exception { + Ignite node = startGrid(0); + + IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1)); + + GridCacheContext cctx = + ((IgniteKernal)node).context().cache().context().cacheContext(CU.cacheId(cache.getName())); + + MvccProcessorImpl crd = mvccProcessor(node); + + // Start query to prevent cleanup. + IgniteInternalFuture fut = crd.requestSnapshotAsync(); + + fut.get(); + + final int KEYS = 1000; + + for (int i = 0; i < 10; i++) { + for (int k = 0; k < KEYS; k++) { + final Integer key = k; + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.put(key, i); + + tx.commit(); + } + } + } + + for (int k = 0; k < KEYS; k++) { + final Integer key = k; + + KeyCacheObject key0 = cctx.toCacheKeyObject(key); + + List> vers = cctx.offheap().mvccAllVersions(cctx, key0); + + assertEquals(10, vers.size()); + + CacheDataRow row = cctx.offheap().read(cctx, key0); + + checkRow(cctx, row, key0, vers.get(0).get1()); + + for (IgniteBiTuple ver : vers) { + MvccVersion cntr = ver.get2(); + + MvccSnapshot readVer = + new MvccSnapshotWithoutTxs(cntr.coordinatorVersion(), cntr.counter(), Integer.MAX_VALUE, 0); + + row = cctx.offheap().mvccRead(cctx, key0, readVer); + + checkRow(cctx, row, key0, ver.get1()); + } + + checkRow(cctx, + cctx.offheap().mvccRead(cctx, key0, version(vers.get(0).get2().coordinatorVersion() + 1, 1)), + key0, + vers.get(0).get1()); + + checkRow(cctx, + cctx.offheap().mvccRead(cctx, key0, version(vers.get(0).get2().coordinatorVersion(), vers.get(0).get2().counter() + 1)), + key0, + vers.get(0).get1()); + + MvccSnapshotResponse ver = version(vers.get(0).get2().coordinatorVersion(), 100000); + + for (int v = 0; v < vers.size(); v++) { + MvccVersion cntr = vers.get(v).get2(); + + ver.addTx(cntr.counter()); + + row = cctx.offheap().mvccRead(cctx, key0, ver); + + if (v == vers.size() - 1) + assertNull(row); + else + checkRow(cctx, row, key0, vers.get(v + 1).get1()); + } + } + + KeyCacheObject key = cctx.toCacheKeyObject(KEYS); + + cache.put(key, 0); + + cache.remove(key); + + cctx.offheap().mvccRemoveAll((GridCacheMapEntry)cctx.cache().entryEx(key)); + + crd.ackQueryDone(fut.get(), MVCC_TRACKER_ID_NA); + } + + /** + * @throws Exception If failed. + */ + public void testExpiration() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-7956"); + final IgniteEx node = startGrid(0); + + IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, 64)); + + final IgniteCache expiryCache = + cache.withExpiryPolicy(new TouchedExpiryPolicy(new Duration(TimeUnit.SECONDS, 1))); + + for (int i = 0; i < 10; i++) + expiryCache.put(1, i); + + assertTrue("Failed to wait for expiration", GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return expiryCache.localPeek(1) == null; + } + }, 5000)); + + for (int i = 0; i < 11; i++) { + if (i % 2 == 0) + expiryCache.put(1, i); + else + expiryCache.remove(1); + } + + assertTrue("Failed to wait for expiration", GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return expiryCache.localPeek(1) == null; + } + }, 5000)); + + expiryCache.put(1, 1); + + assertTrue("Failed to wait for expiration", GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + try { + GridCacheContext cctx = node.context().cache().context().cacheContext(CU.cacheId(DEFAULT_CACHE_NAME)); + + KeyCacheObject key = cctx.toCacheKeyObject(1); + + return cctx.offheap().read(cctx, key) == null; + } + catch (Exception e) { + fail(); + + return false; + } + } + }, 5000)); + } + + /** + * @throws Exception If failed. + */ + public void testChangeExpireTime() throws Exception { + final IgniteEx node = startGrid(0); + + IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, 64)); + + cache.put(1, 1); + + final IgniteCache expiryCache = + cache.withExpiryPolicy(new TouchedExpiryPolicy(new Duration(TimeUnit.SECONDS, 1))); + + expiryCache.get(1); + } + + /** + * @param cctx Context. + * @param row Row. + * @param expKey Expected row key. + * @param expVal Expected row value. + */ + private void checkRow(GridCacheContext cctx, CacheDataRow row, KeyCacheObject expKey, Object expVal) { + assertNotNull(row); + assertEquals(expKey, row.key()); + assertEquals(expVal, row.value().value(cctx.cacheObjectContext(), false)); + } + + /** + * @param crdVer Coordinator version. + * @param cntr Counter. + * @return Version. + */ + private MvccSnapshotResponse version(long crdVer, long cntr) { + MvccSnapshotResponse res = new MvccSnapshotResponse(); + + res.init(0, crdVer, cntr, MvccUtils.MVCC_START_OP_CNTR, MvccUtils.MVCC_COUNTER_NA, 0); + + return res; + } + + /** + * @param ccfg Cache configuration. + */ + private void logCacheInfo(CacheConfiguration ccfg) { + log.info("Test cache [mode=" + ccfg.getCacheMode() + + ", sync=" + ccfg.getWriteSynchronizationMode() + + ", backups=" + ccfg.getBackups() + + ", near=" + (ccfg.getNearConfiguration() != null) + + ']'); + } + + /** + * @param cache Cache. + * @return Test keys. + * @throws Exception If failed. + */ + private List testKeys(IgniteCache cache) throws Exception { + CacheConfiguration ccfg = cache.getConfiguration(CacheConfiguration.class); + + List keys = new ArrayList<>(); + + if (ccfg.getCacheMode() == PARTITIONED) + keys.add(nearKey(cache)); + + keys.add(primaryKey(cache)); + + if (ccfg.getBackups() != 0) + keys.add(backupKey(cache)); + + return keys; + } + + /** + * Checks values obtained with different read modes. + * And returns value in case of it's equality for all read modes. + * Do not use in tests with writers contention. + * + * // TODO remove inTx flag in IGNITE-6938 + * @param inTx Flag whether current read is inside transaction. + * This is because reads can't see writes made in current transaction. + * @param cache Cache. + * @param key Key. + * @param readModes Read modes to check. + * @return Value. + */ + private Object checkAndGet(boolean inTx, IgniteCache cache, Object key, ReadMode ... readModes) { + assert readModes != null && readModes.length > 0; + + if (inTx) + return getByReadMode(inTx, cache, key, GET); + + Object prevVal = null; + + for (int i = 0; i < readModes.length; i++) { + ReadMode readMode = readModes[i]; + + Object curVal = getByReadMode(inTx, cache, key, readMode); + + if (i == 0) + prevVal = curVal; + else { + assertEquals("Different results on " + readModes[i - 1].name() + " and " + + readMode.name() + " read modes.", prevVal, curVal); + + prevVal = curVal; + } + } + + return prevVal; + } + + /** + * Reads value from cache for the given key using given read mode. + * + * // TODO IGNITE-6938 remove inTx flag + * // TODO IGNITE-6739 add SQL-get support "select _key, _val from cache where _key = key" + * @param inTx Flag whether current read is inside transaction. + * This is because reads can't see writes made in current transaction. + * @param cache Cache. + * @param key Key. + * @param readMode Read mode. + * @return Value. + */ + private Object getByReadMode(boolean inTx, IgniteCache cache, final Object key, ReadMode readMode) { + + // TODO Remove in IGNITE-6938 + if (inTx) + readMode = GET; + + switch (readMode) { + case GET: + return cache.get(key); + + case SCAN: + List res = cache.query(new ScanQuery(new IgniteBiPredicate() { + @Override public boolean apply(Object k, Object v) { + return k.equals(key); + } + })).getAll(); + + assertTrue(res.size() <= 1); + + return res.isEmpty() ? null : ((IgniteBiTuple)res.get(0)).getValue(); + + default: + throw new IgniteException("Unsupported read mode: " + readMode); + } + } + + + /** + * Checks values obtained with different read modes. + * And returns value in case of it's equality for all read modes. + * Do not use in tests with writers contention. + * + * // TODO remove inTx flag in IGNITE-7764 + * @param inTx Flag whether current read is inside transaction. + * This is because reads can't see writes made in current transaction. + * @param cache Cache. + * @param keys Key. + * @param readModes Read modes to check. + * @return Value. + */ + private Map checkAndGetAll(boolean inTx, IgniteCache cache, Set keys, ReadMode ... readModes) { + assert readModes != null && readModes.length > 0; + + if (inTx) + return getAllByReadMode(inTx, cache, keys, GET); + + Map prevVal = null; + + for (int i = 0; i < readModes.length; i++) { + ReadMode readMode = readModes[i]; + + Map curVal = getAllByReadMode(inTx, cache, keys, readMode); + + if (i == 0) + prevVal = curVal; + else { + assertEquals("Different results on read modes " + readModes[i - 1] + " and " + + readMode.name(), prevVal, curVal); + + prevVal = curVal; + } + } + + return prevVal; + } + + + /** + * Reads value from cache for the given key using given read mode. + * + * // TODO IGNITE-7764 remove inTx flag + * // TODO IGNITE-6739 add SQL-get support "select _key, _val from cache where _key in ... keySet" + * @param inTx Flag whether current read is inside transaction. + * This is because reads can't see writes made in current transaction. + * @param cache Cache. + * @param keys Key. + * @param readMode Read mode. + * @return Value. + */ + private Map getAllByReadMode(boolean inTx, IgniteCache cache, Set keys, ReadMode readMode) { + + // TODO Remove in IGNITE-6938 + if (inTx) + readMode = GET; + + switch (readMode) { + case GET: + return cache.getAll(keys); + + case SCAN: + Map res = (Map)cache.query(new ScanQuery(new IgniteBiPredicate() { + @Override public boolean apply(Object k, Object v) { + return keys.contains(k); + } + })).getAll() + .stream() + .collect(Collectors.toMap(v -> ((IgniteBiTuple)v).getKey(), v -> ((IgniteBiTuple)v).getValue())); + + assertTrue(res.size() <= keys.size()); + + return res; + + default: + throw new IgniteException("Unsupported read mode: " + readMode); + } + } + + /** + * + */ + static class Value { + /** */ + int key; + + /** */ + int cnt; + + /** + * @param key Key. + * @param cnt Update count. + */ + Value(int key, int cnt) { + this.key = key; + this.cnt = cnt; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(Value.class, this); + } + } + + /** + * + */ + static class TestKey implements Serializable { + /** */ + private final int key; + + /** */ + private final byte[] payload; + + /** + * @param key Key. + * @param payloadSize Payload size. + */ + public TestKey(int key, int payloadSize) { + this.key = key; + this.payload = new byte[payloadSize]; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + TestKey testKey = (TestKey)o; + + if (key != testKey.key) + return false; + + return Arrays.equals(payload, testKey.payload); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = key; + + res = 31 * res + Arrays.hashCode(payload); + + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "TestKey [k=" + key + ", payloadLen=" + payload.length + ']'; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java new file mode 100644 index 0000000000000..cba6d990a4310 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.testframework.GridTestUtils; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * Vacuum test. + */ +public class CacheMvccVacuumTest extends CacheMvccAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return PARTITIONED; + } + + /** + * @throws Exception If failed. + */ + public void testStartStopVacuumInMemory() throws Exception { + Ignite node0 = startGrid(0); + Ignite node1 = startGrid(1); + + ensureVacuum(node0); + ensureVacuum(node1); + + stopGrid(0); + + ensureNoVacuum(node0); + ensureVacuum(node1); + + stopGrid(1); + + ensureNoVacuum(node0); + ensureNoVacuum(node1); + } + + /** + * @throws Exception If failed. + */ + public void testStartStopVacuumPersistence() throws Exception { + persistence = true; + + Ignite node0 = startGrid(0); + Ignite node1 = startGrid(1); + + ensureNoVacuum(node0); + ensureNoVacuum(node1); + + node1.cluster().active(true); + + ensureVacuum(node0); + ensureVacuum(node1); + + node1.cluster().active(false); + + ensureNoVacuum(node0); + ensureNoVacuum(node1); + + node1.cluster().active(true); + + ensureVacuum(node0); + ensureVacuum(node1); + + stopGrid(0); + + ensureNoVacuum(node0); + ensureVacuum(node1); + + stopGrid(1); + + ensureNoVacuum(node0); + ensureNoVacuum(node1); + } + + /** + * @throws Exception If failed. + */ + public void testVacuumNotStartedWithoutMvcc() throws Exception { + IgniteConfiguration cfg = getConfiguration("grid1").setMvccEnabled(false); + + Ignite node = startGrid(cfg); + + ensureNoVacuum(node); + } + + /** + * @throws Exception If failed. + */ + public void testVacuumNotStartedWithoutMvccPersistence() throws Exception { + persistence = true; + + IgniteConfiguration cfg = getConfiguration("grid1").setMvccEnabled(false); + + Ignite node = startGrid(cfg); + + ensureNoVacuum(node); + + node.cluster().active(true); + + ensureNoVacuum(node); + } + + /** + * Ensures vacuum is running on the given node. + * + * @param node Node. + */ + private void ensureVacuum(Ignite node) { + MvccProcessorImpl crd = mvccProcessor(node); + + assertNotNull(crd); + + List vacuumWorkers = GridTestUtils.getFieldValue(crd, "vacuumWorkers"); + + assertNotNull(vacuumWorkers); + assertFalse(vacuumWorkers.isEmpty()); + + for (GridWorker w : vacuumWorkers) { + assertFalse(w.isCancelled()); + assertFalse(w.isDone()); + } + } + + /** + * Ensures vacuum is stopped on the given node. + * + * @param node Node. + */ + private void ensureNoVacuum(Ignite node) { + MvccProcessorImpl crd = mvccProcessor(node); + + if (crd != null) + assertNull(GridTestUtils.>getFieldValue(crd, "vacuumWorkers")); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicyInitializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicyInitializationTest.java index 774fcc68a61cf..ccf15027a4998 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicyInitializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicyInitializationTest.java @@ -75,7 +75,7 @@ public void testNoConfigProvided() throws Exception { Collection allMemPlcs = ignite.context().cache().context().database().dataRegions(); - assertTrue(allMemPlcs.size() == 2); + assertEquals(2, allMemPlcs.size()); verifyDefaultAndSystemMemoryPolicies(allMemPlcs); } @@ -91,7 +91,7 @@ public void testCustomConfigNoDefault() throws Exception { Collection allMemPlcs = ignite.context().cache().context().database().dataRegions(); - assertTrue(allMemPlcs.size() == 3); + assertEquals(3, allMemPlcs.size()); verifyDefaultAndSystemMemoryPolicies(allMemPlcs); @@ -112,13 +112,13 @@ public void testCustomConfigOverridesDefault() throws Exception { Collection allMemPlcs = dbMgr.dataRegions(); - assertTrue(allMemPlcs.size() == 2); + assertEquals(2, allMemPlcs.size()); verifyDefaultAndSystemMemoryPolicies(allMemPlcs); DataRegion dfltMemPlc = U.field(dbMgr, "dfltDataRegion"); - assertTrue(dfltMemPlc.config().getMaxSize() == USER_DEFAULT_MEM_PLC_SIZE); + assertEquals(dfltMemPlc.config().getMaxSize(), USER_DEFAULT_MEM_PLC_SIZE); } /** @@ -136,13 +136,13 @@ public void testCustomConfigOverridesDefaultNameAndDeclaresDefault() throws Exce Collection allMemPlcs = dbMgr.dataRegions(); - assertTrue(allMemPlcs.size() == 3); + assertEquals(3, allMemPlcs.size()); verifyDefaultAndSystemMemoryPolicies(allMemPlcs); DataRegion dfltMemPlc = U.field(dbMgr, "dfltDataRegion"); - assertTrue(dfltMemPlc.config().getMaxSize() == USER_CUSTOM_MEM_PLC_SIZE); + assertEquals(dfltMemPlc.config().getMaxSize(), USER_CUSTOM_MEM_PLC_SIZE); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java index c61b3c0a39a2d..0f219523cc5e1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.AllocatedPageTracker; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteFuture; @@ -53,6 +54,12 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager { // No-op. } + /** {@inheritDoc} */ + @Override public void initialize(int cacheId, int partitions, String workingDir, + AllocatedPageTracker tracker) throws IgniteCheckedException { + // No-op. + } + /** {@inheritDoc} */ @Override public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cacheData) throws IgniteCheckedException { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java index 3c368f747ba60..487cdbe25bd97 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java @@ -28,12 +28,14 @@ import java.util.Random; import java.util.Set; import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -249,6 +251,53 @@ public void testRetries() throws IgniteCheckedException { } } + /** + * @throws IgniteCheckedException If failed. + */ + public void testFindWithClosure() throws IgniteCheckedException { + TestTree tree = createTestTree(true); + TreeMap map = new TreeMap<>(); + + long size = CNT * CNT; + + for (long i = 1; i <= size; i++) { + tree.put(i); + map.put(i, i); + } + + checkCursor(tree.find(null, null, new TestTreeFindFilteredClosure(Collections.emptySet()), null), + Collections.emptyList().iterator()); + + checkCursor(tree.find(null, null, new TestTreeFindFilteredClosure(map.keySet()), null), + map.values().iterator()); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 100; i++) { + Long val = rnd.nextLong(size) + 1; + + checkCursor(tree.find(null, null, new TestTreeFindFilteredClosure(Collections.singleton(val)), null), + Collections.singleton(val).iterator()); + } + + for (int i = 0; i < 200; i++) { + long vals = rnd.nextLong(size) + 1; + + TreeSet exp = new TreeSet<>(); + + for (long k = 0; k < vals; k++) + exp.add(rnd.nextLong(size) + 1); + + checkCursor(tree.find(null, null, new TestTreeFindFilteredClosure(exp), null), exp.iterator()); + + checkCursor(tree.find(0L, null, new TestTreeFindFilteredClosure(exp), null), exp.iterator()); + + checkCursor(tree.find(0L, size, new TestTreeFindFilteredClosure(exp), null), exp.iterator()); + + checkCursor(tree.find(null, size, new TestTreeFindFilteredClosure(exp), null), exp.iterator()); + } + } + /** * @throws IgniteCheckedException If failed. */ @@ -605,6 +654,7 @@ private void doTestPutRemove(boolean canGetRow) throws IgniteCheckedException { assertNoLocks(); assertEquals(x, tree.findOne(x).longValue()); + checkIterate(tree, x, x, x, true); assertNoLocks(); @@ -619,12 +669,15 @@ private void doTestPutRemove(boolean canGetRow) throws IgniteCheckedException { assertNull(tree.findOne(-1L)); - for (long x = 0; x < cnt; x++) + for (long x = 0; x < cnt; x++) { assertEquals(x, tree.findOne(x).longValue()); + checkIterate(tree, x, x, x, true); + } assertNoLocks(); assertNull(tree.findOne(cnt)); + checkIterate(tree, cnt, cnt, null, false); for (long x = RMV_INC > 0 ? 0 : cnt - 1; x >= 0 && x < cnt; x += RMV_INC) { X.println(" -- " + x); @@ -638,6 +691,7 @@ private void doTestPutRemove(boolean canGetRow) throws IgniteCheckedException { assertNoLocks(); assertNull(tree.findOne(x)); + checkIterate(tree, x, x, null, false); assertNoLocks(); @@ -653,6 +707,40 @@ private void doTestPutRemove(boolean canGetRow) throws IgniteCheckedException { assertNoLocks(); } + /** + * @param tree Tree. + * @param lower Lower bound. + * @param upper Upper bound. + * @param exp Value to find. + * @param expFound {@code True} if value should be found. + * @throws IgniteCheckedException If failed. + */ + private void checkIterate(TestTree tree, long lower, long upper, Long exp, boolean expFound) + throws IgniteCheckedException { + TestTreeRowClosure c = new TestTreeRowClosure(exp); + + tree.iterate(lower, upper, c); + + assertEquals(expFound, c.found); + } + + /** + * @param tree Tree. + * @param lower Lower bound. + * @param upper Upper bound. + * @param c Closure. + * @param expFound {@code True} if value should be found. + * @throws IgniteCheckedException If failed. + */ + private void checkIterateC(TestTree tree, long lower, long upper, TestTreeRowClosure c, boolean expFound) + throws IgniteCheckedException { + c.found = false; + + tree.iterate(lower, upper, c); + + assertEquals(expFound, c.found); + } + /** * @throws IgniteCheckedException If failed. */ @@ -2057,7 +2145,209 @@ public void testFindFirstAndLast() throws IgniteCheckedException { } /** - * + * @throws Exception If failed. + */ + public void testIterate() throws Exception { + MAX_PER_PAGE = 5; + + TestTree tree = createTestTree(true); + + checkIterate(tree, 0L, 100L, null, false); + + for (long idx = 1L; idx <= 10L; ++idx) + tree.put(idx); + + for (long idx = 1L; idx <= 10L; ++idx) + checkIterate(tree, idx, 100L, idx, true); + + checkIterate(tree, 0L, 100L, 1L, true); + + for (long idx = 1L; idx <= 10L; ++idx) + checkIterate(tree, idx, 100L, 10L, true); + + checkIterate(tree, 0L, 100L, 100L, false); + + for (long idx = 1L; idx <= 10L; ++idx) + checkIterate(tree, 0L, 100L, idx, true); + + for (long idx = 0L; idx <= 10L; ++idx) + checkIterate(tree, idx, 11L, -1L, false); + } + + /** + * @throws Exception If failed. + */ + public void testIterateConcurrentPutRemove() throws Exception { + iterateConcurrentPutRemove(); + } + + /** + * @throws Exception If failed. + */ + public void testIterateConcurrentPutRemove_1() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-7265"); + + MAX_PER_PAGE = 1; + + iterateConcurrentPutRemove(); + } + + /** + * @throws Exception If failed. + */ + public void testIterateConcurrentPutRemove_5() throws Exception { + MAX_PER_PAGE = 5; + + iterateConcurrentPutRemove(); + } + + /** + * @throws Exception If failed. + */ + public void testIteratePutRemove_10() throws Exception { + MAX_PER_PAGE = 10; + + iterateConcurrentPutRemove(); + } + + /** + * @throws Exception If failed. + */ + private void iterateConcurrentPutRemove() throws Exception { + final TestTree tree = createTestTree(true); + + final int KEYS = 10_000; + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 10; i++) { + for (long idx = 0L; idx < KEYS; ++idx) + tree.put(idx); + + final Long findKey; + + if (MAX_PER_PAGE > 0) { + switch (i) { + case 0: + findKey = 1L; + + break; + + case 1: + findKey = (long)MAX_PER_PAGE; + + break; + + case 2: + findKey = (long)MAX_PER_PAGE - 1; + + break; + + case 3: + findKey = (long)MAX_PER_PAGE + 1; + + break; + + case 4: + findKey = (long)(KEYS / MAX_PER_PAGE / 2) * MAX_PER_PAGE; + + break; + + case 5: + findKey = (long)(KEYS / MAX_PER_PAGE / 2) * MAX_PER_PAGE - 1; + + break; + + case 6: + findKey = (long)(KEYS / MAX_PER_PAGE / 2) * MAX_PER_PAGE + 1; + + break; + + case 7: + findKey = (long)KEYS - 1; + + break; + + default: + findKey = rnd.nextLong(KEYS); + } + } + else + findKey = rnd.nextLong(KEYS); + + info("Iteration [iter=" + i + ", key=" + findKey + ']'); + + assertEquals(findKey, tree.findOne(findKey)); + checkIterate(tree, findKey, findKey, findKey, true); + + IgniteInternalFuture getFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + TestTreeRowClosure p = new TestTreeRowClosure(findKey); + + TestTreeRowClosure falseP = new TestTreeRowClosure(-1L); + + int cnt = 0; + + while (!stop.get()) { + int shift = MAX_PER_PAGE > 0 ? rnd.nextInt(MAX_PER_PAGE * 2) : rnd.nextInt(100); + + checkIterateC(tree, findKey, findKey, p, true); + + checkIterateC(tree, findKey - shift, findKey, p, true); + + checkIterateC(tree, findKey - shift, findKey + shift, p, true); + + checkIterateC(tree, findKey, findKey + shift, p, true); + + checkIterateC(tree, -100L, KEYS + 100L, falseP, false); + + cnt++; + } + + info("Done, read count: " + cnt); + + return null; + } + }, 10, "find"); + + asyncRunFut = new GridCompoundFuture<>(); + + asyncRunFut.add(getFut); + + asyncRunFut.markInitialized(); + + try { + U.sleep(100); + + for (int j = 0; j < 20; j++) { + for (long idx = 0L; idx < KEYS / 2; ++idx) { + long toRmv = rnd.nextLong(KEYS); + + if (toRmv != findKey) + tree.remove(toRmv); + } + + for (long idx = 0L; idx < KEYS / 2; ++idx) { + long put = rnd.nextLong(KEYS); + + tree.put(put); + } + } + } + finally { + stop.set(true); + } + + asyncRunFut.get(); + + stop.set(false); + } + } + + /** + * @throws Exception If failed. */ public void testConcurrentGrowDegenerateTreeAndConcurrentRemove() throws Exception { //calculate tree size when split happens @@ -2264,6 +2554,17 @@ else if (op == 3) { last = c.get(); } + + TestTreeFindFirstClosure cl = new TestTreeFindFirstClosure(); + + tree.iterate((long)low, (long)high, cl); + + last = cl.val; + + if (last != null) { + assertTrue(low + " <= " + last + " <= " + high, last >= low); + assertTrue(low + " <= " + last + " <= " + high, last <= high); + } } return null; @@ -2401,7 +2702,7 @@ public TestTree(ReuseList reuseList, boolean canGetRow, int cacheId, PageMemory } /** {@inheritDoc} */ - @Override protected Long getRow(BPlusIO io, long pageAddr, int idx, Object ignore) + @Override public Long getRow(BPlusIO io, long pageAddr, int idx, Object ignore) throws IgniteCheckedException { assert io.canGetRow() : io; @@ -2688,4 +2989,74 @@ private static final class LongLeafIO extends BPlusLeafIO { return PageUtils.getLong(pageAddr, offset(idx)); } } + + /** + * + */ + static class TestTreeRowClosure implements BPlusTree.TreeRowClosure { + /** */ + private final Long expVal; + + /** */ + private boolean found; + + /** + * @param expVal Value to find or {@code null} to find first. + */ + TestTreeRowClosure(Long expVal) { + this.expVal = expVal; + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, int idx) + throws IgniteCheckedException { + assert !found; + + found = expVal == null || io.getLookupRow(tree, pageAddr, idx).equals(expVal); + + return !found; + } + } + + /** + * + */ + static class TestTreeFindFirstClosure implements BPlusTree.TreeRowClosure { + /** */ + private Long val; + + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, int idx) + throws IgniteCheckedException { + assert val == null; + + val = io.getLookupRow(tree, pageAddr, idx); + + return false; + } + } + + /** + * + */ + static class TestTreeFindFilteredClosure implements BPlusTree.TreeRowClosure { + /** */ + private final Set vals; + + /** + * @param vals Values to allow in filter. + */ + TestTreeFindFilteredClosure(Set vals) { + this.vals = vals; + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, int idx) + throws IgniteCheckedException { + Long val = io.getLookupRow(tree, pageAddr, idx); + + return vals.contains(val); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java index 80daff29633fe..17aa9b0595832 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.persistence.evict.NoOpPageEvictionTracker; import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl; import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; @@ -406,6 +407,20 @@ private TestDataRow(int keySize, int valSize) { return 0; } + /** {@inheritDoc} */ + @Override public int size() throws IgniteCheckedException { + int len = key().valueBytesLength(null); + + len += value().valueBytesLength(null) + CacheVersionIO.size(version(), false) + 8; + + return len + (cacheId() != 0 ? 4 : 0); + } + + /** {@inheritDoc} */ + @Override public int headerSize() { + return 0; + } + /** {@inheritDoc} */ @Override public long link() { return link; @@ -425,6 +440,46 @@ private TestDataRow(int keySize, int valSize) { @Override public int cacheId() { return 0; } + + /** {@inheritDoc} */ + @Override public long newMvccCoordinatorVersion() { + return 0; + } + + /** {@inheritDoc} */ + @Override public long newMvccCounter() { + return 0; + } + + /** {@inheritDoc} */ + @Override public int newMvccOperationCounter() { + return 0; + } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return 0; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return 0; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return 0; + } + + /** {@inheritDoc} */ + @Override public byte mvccTxState() { + return 0; + } + + /** {@inheritDoc} */ + @Override public byte newMvccTxState() { + return 0; + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSeflTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSeflTest.java new file mode 100644 index 0000000000000..abe6dee696641 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSeflTest.java @@ -0,0 +1,68 @@ +/* + * 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.datastreamer; + +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; + +/** */ +public class DataStreamProcessorMvccSeflTest extends DataStreamProcessorSelfTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration igniteConfiguration = super.getConfiguration(igniteInstanceName); + + CacheConfiguration[] cacheConfigurations = igniteConfiguration.getCacheConfiguration(); + + assert cacheConfigurations == null || cacheConfigurations.length == 0 + || (cacheConfigurations.length == 1 && cacheConfigurations[0].getAtomicityMode() == TRANSACTIONAL); + + igniteConfiguration.setMvccEnabled(true); + + return igniteConfiguration; + } + + /** {@inheritDoc} */ + @Override public void testPartitioned() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8149"); + + super.testPartitioned(); + } + + /** {@inheritDoc} */ + @Override public void testColocated() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8149"); + + super.testColocated(); + } + + /** {@inheritDoc} */ + @Override public void testReplicated() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8149"); + + super.testReplicated(); + } + + /** {@inheritDoc} */ + @Override public void testUpdateStore() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8582"); + + super.testUpdateStore(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java index 91345fe3e7b4e..7075973819a27 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java @@ -371,11 +371,27 @@ private void checkIsolatedDataStreamer() throws Exception { if (aff.isPrimary(locNode, key) || aff.isBackup(locNode, key)) { GridCacheEntryEx entry = cache0.entryEx(key); - entry.unswap(); + try { + // lock non obsolete entry + while (true) { + entry.lockEntry(); + + if (!entry.obsolete()) + break; + + entry.unlockEntry(); + + entry = cache0.entryEx(key); + } - assertNotNull("Missing entry for key: " + key, entry); - assertEquals(new Integer((key < 100 ? -1 : key)), - CU.value(entry.rawGet(), cache0.context(), false)); + entry.unswap(); + + assertEquals(new Integer((key < 100 ? -1 : key)), + CU.value(entry.rawGet(), cache0.context(), false)); + } + finally { + entry.unlockEntry(); + } } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserTransactionalKeywordsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserTransactionalKeywordsSelfTest.java new file mode 100644 index 0000000000000..103bb97925cc6 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserTransactionalKeywordsSelfTest.java @@ -0,0 +1,96 @@ +/* + * 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.sql; + +import org.apache.ignite.internal.sql.command.SqlBeginTransactionCommand; +import org.apache.ignite.internal.sql.command.SqlCommand; +import org.apache.ignite.internal.sql.command.SqlCommitTransactionCommand; +import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand; + +/** + * Tests for processing of keywords BEGIN, COMMIT, ROLLBACK, START. + */ +public class SqlParserTransactionalKeywordsSelfTest extends SqlParserAbstractSelfTest { + /** + * Test parsing of different forms of BEGIN/START. + */ + public void testBegin() { + assertBegin("begin"); + assertBegin("BEGIN"); + assertBegin("BEGIN work"); + assertBegin("begin Transaction"); + assertBegin("StarT TransactioN"); + + assertParseError(null, "begin index", "Unexpected token: \"INDEX\""); + assertParseError(null, "start work", "Unexpected token: \"WORK\" (expected: \"TRANSACTION\")"); + assertParseError(null, "start", "Unexpected end of command (expected: \"TRANSACTION\")"); + } + + /** + * Test parsing of different forms of COMMIT. + */ + public void testCommit() { + assertCommit("commit"); + assertCommit("COMMIT transaction"); + + assertParseError(null, "commit index", "Unexpected token: \"INDEX\""); + } + + /** + * Test parsing of different forms of ROLLBACK. + */ + public void testRollback() { + assertRollback("rollback"); + assertRollback("ROLLBACK transaction"); + + assertParseError(null, "rollback index", "Unexpected token: \"INDEX\""); + } + + /** + * Test that given SQL is parsed as a BEGIN command. + * @param sql command. + */ + private static void assertBegin(String sql) { + assertTrue(parse(sql) instanceof SqlBeginTransactionCommand); + } + + /** + * Test that given SQL is parsed as a BEGIN command. + * @param sql command. + */ + private static void assertCommit(String sql) { + assertTrue(parse(sql) instanceof SqlCommitTransactionCommand); + } + + /** + * Test that given SQL is parsed as a BEGIN command. + * @param sql command. + */ + private static void assertRollback(String sql) { + assertTrue(parse(sql) instanceof SqlRollbackTransactionCommand); + } + + /** + * Parse single SQL command. + * @param sql command. + * @return parsed command. + */ + private static SqlCommand parse(String sql) { + return new SqlParser(null, sql).nextCommand(); + } +} 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 341f7be777c2d..09475e6808692 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 @@ -1582,6 +1582,14 @@ public String getTestIgniteInstanceName(int idx) { return getTestIgniteInstanceName() + idx; } + /** + * @param idx Index of the Ignite instance. + * @return Indexed Ignite instance name. + */ + protected String testNodeName(int idx) { + return getTestIgniteInstanceName(idx); + } + /** * Parses test Ignite instance index from test Ignite instance name. * diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java new file mode 100644 index 0000000000000..b5275b9dba833 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.testsuites; + +import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccClusterRestartTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccConfigurationValidationTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccIteratorWithConcurrentTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccLocalEntriesWithConcurrentTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccOperationChecksTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedCoordinatorFailoverTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccProcessorTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedCoordinatorFailoverTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccScanQueryWithConcurrentTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTransactionsTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccVacuumTest; +import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorMvccSeflTest; + +/** + * + */ +public class IgniteCacheMvccTestSuite extends TestSuite { + /** + * @return Test suite. + */ + public static TestSuite suite() { + TestSuite suite = new TestSuite("IgniteCache MVCC Test Suite"); + + suite.addTestSuite(CacheMvccTransactionsTest.class); + suite.addTestSuite(CacheMvccProcessorTest.class); + suite.addTestSuite(CacheMvccClusterRestartTest.class); + suite.addTestSuite(CacheMvccConfigurationValidationTest.class); + suite.addTestSuite(CacheMvccOperationChecksTest.class); + suite.addTestSuite(CacheMvccIteratorWithConcurrentTransactionTest.class); + suite.addTestSuite(CacheMvccLocalEntriesWithConcurrentTransactionTest.class); + suite.addTestSuite(CacheMvccScanQueryWithConcurrentTransactionTest.class); + suite.addTestSuite(CacheMvccSizeWithConcurrentTransactionTest.class); + suite.addTestSuite(CacheMvccVacuumTest.class); + suite.addTestSuite(CacheMvccPartitionedCoordinatorFailoverTest.class); + suite.addTestSuite(CacheMvccReplicatedCoordinatorFailoverTest.class); + suite.addTestSuite(DataStreamProcessorMvccSeflTest.class); + + return suite; + } +} diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java index 21a0c618bf279..ca144abd89d69 100644 --- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java +++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/IgniteWalConverter.java @@ -29,6 +29,8 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2ExtrasLeafIO; import org.apache.ignite.internal.processors.query.h2.database.io.H2InnerIO; import org.apache.ignite.internal.processors.query.h2.database.io.H2LeafIO; +import org.apache.ignite.internal.processors.query.h2.database.io.H2MvccInnerIO; +import org.apache.ignite.internal.processors.query.h2.database.io.H2MvccLeafIO; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.logger.NullLogger; import org.jetbrains.annotations.Nullable; @@ -39,6 +41,7 @@ public class IgniteWalConverter { /** * @param args Args. + * @throws Exception If failed. */ public static void main(String[] args) throws Exception { if (args.length < 2) @@ -47,7 +50,7 @@ public static void main(String[] args) throws Exception { "\t2. Path to dir with wal files.\n" + "\t3. (Optional) Path to dir with archive wal files."); - PageIO.registerH2(H2InnerIO.VERSIONS, H2LeafIO.VERSIONS); + PageIO.registerH2(H2InnerIO.VERSIONS, H2LeafIO.VERSIONS, H2MvccInnerIO.VERSIONS, H2MvccLeafIO.VERSIONS); H2ExtrasInnerIO.register(); H2ExtrasLeafIO.register(); diff --git a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java index b4a8af49adf14..831e6749128c7 100644 --- a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java +++ b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.processors.query.h2.opt; -import com.vividsolutions.jts.geom.Envelope; -import com.vividsolutions.jts.geom.Geometry; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -28,6 +26,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.vividsolutions.jts.geom.Envelope; +import com.vividsolutions.jts.geom.Geometry; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.query.h2.H2Cursor; import org.apache.ignite.internal.util.GridCursorIteratorWrapper; @@ -340,7 +340,12 @@ private GridCursor rowIterator(Iterator i, TableFilter fi long time = System.currentTimeMillis(); - IndexingQueryFilter qryFilter = threadLocalFilter(); + IndexingQueryFilter qryFilter = null; + GridH2QueryContext qctx = GridH2QueryContext.get(); + + if (qctx != null) { + qryFilter = qctx.filter(); + } IndexingQueryCacheFilter qryCacheFilter = qryFilter != null ? qryFilter.forCache(getTable().cacheName()) : null; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java index 4a93aafd59558..a5f0ca20b5260 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java @@ -65,6 +65,12 @@ public class GridCacheTwoStepQuery { /** */ private CacheQueryPartitionInfo[] derivedPartitions; + /** */ + private boolean mvccEnabled; + + /** {@code FOR UPDATE} flag. */ + private boolean forUpdate; + /** * @param originalSql Original query SQL. * @param tbls Tables in query. @@ -241,6 +247,8 @@ public GridCacheTwoStepQuery copy() { cp.distributedJoins = distributedJoins; cp.derivedPartitions = derivedPartitions; cp.local = local; + cp.mvccEnabled = mvccEnabled; + cp.forUpdate = forUpdate; for (int i = 0; i < mapQrys.size(); i++) cp.mapQrys.add(mapQrys.get(i).copy()); @@ -262,6 +270,34 @@ public Set tables() { return tbls; } + /** + * @return Mvcc flag. + */ + public boolean mvccEnabled() { + return mvccEnabled; + } + + /** + * @param mvccEnabled Mvcc flag. + */ + public void mvccEnabled(boolean mvccEnabled) { + this.mvccEnabled = mvccEnabled; + } + + /** + * @return {@code FOR UPDATE} flag. + */ + public boolean forUpdate() { + return forUpdate; + } + + /** + * @param forUpdate {@code FOR UPDATE} flag. + */ + public void forUpdate(boolean forUpdate) { + this.forUpdate = forUpdate; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridCacheTwoStepQuery.class, this); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DhtResultSetEnlistFuture.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DhtResultSetEnlistFuture.java new file mode 100644 index 0000000000000..1d382f7052412 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DhtResultSetEnlistFuture.java @@ -0,0 +1,63 @@ +/* + * 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.sql.ResultSet; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxAbstractEnlistFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class DhtResultSetEnlistFuture extends GridDhtTxAbstractEnlistFuture implements ResultSetEnlistFuture { + /** */ + private ResultSet rs; + + /** + * @param nearNodeId Near node ID. + * @param nearLockVer Near lock version. + * @param mvccSnapshot Mvcc snapshot. + * @param threadId Thread ID. + * @param nearFutId Near future id. + * @param nearMiniId Near mini future id. + * @param parts Partitions. + * @param tx Transaction. + * @param timeout Lock acquisition timeout. + * @param cctx Cache context. + * @param rs Result set to process. + */ + public DhtResultSetEnlistFuture(UUID nearNodeId, GridCacheVersion nearLockVer, + MvccSnapshot mvccSnapshot, long threadId, IgniteUuid nearFutId, int nearMiniId, @Nullable int[] parts, + GridDhtTxLocalAdapter tx, long timeout, GridCacheContext cctx, ResultSet rs) { + super(nearNodeId, nearLockVer, mvccSnapshot, threadId, nearFutId, nearMiniId, parts, tx, timeout, cctx); + + this.rs = rs; + } + + /** {@inheritDoc} */ + @Override protected UpdateSourceIterator createIterator() { + return ResultSetEnlistFuture.createIterator(rs); + } +} 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 5270e7ff1736a..6ce43ddc3675a 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 @@ -43,6 +43,8 @@ import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; 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.bulkload.BulkLoadAckClientParameters; import org.apache.ignite.internal.processors.bulkload.BulkLoadCacheWriter; import org.apache.ignite.internal.processors.bulkload.BulkLoadParser; @@ -51,14 +53,20 @@ import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.StaticMvccQueryTracker; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx; import org.apache.ignite.internal.processors.odbc.SqlStateCode; +import org.apache.ignite.internal.processors.query.EnlistOperation; import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldsResult; import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter; import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender; import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo; import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils; @@ -67,6 +75,7 @@ import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap; @@ -86,6 +95,11 @@ import org.h2.command.dml.Update; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.checkActive; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccTracker; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.requestSnapshot; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.tx; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.txStart; import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.DUPLICATE_KEY; import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.createJdbcSqlException; import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META; @@ -95,7 +109,7 @@ */ public class DmlStatementsProcessor { /** Default number of attempts to re-run DELETE and UPDATE queries in case of concurrent modifications of values. */ - private final static int DFLT_DML_RERUN_ATTEMPTS = 4; + private static final int DFLT_DML_RERUN_ATTEMPTS = 4; /** Indexing. */ private IgniteH2Indexing idx; @@ -168,7 +182,7 @@ private UpdateResult updateSqlFields(String schemaName, Connection conn, Prepare UpdateResult r; try { - r = executeUpdateStatement(schemaName, cctx, conn, prepared, fieldsQry, loc, filters, cancel); + r = executeUpdateStatement(schemaName, plan, fieldsQry, loc, filters, cancel); } finally { cctx.operationContextPerCall(opCtx); @@ -211,17 +225,16 @@ private Collection updateSqlFieldsBatched(String schemaName, Conne UpdatePlan plan = getPlanForStatement(schemaName, conn, prepared, fieldsQry, loc, null); - if (plan.hasRows() && plan.mode() == UpdateMode.INSERT) { - GridCacheContext cctx = plan.cacheContext(); + GridCacheContext cctx = plan.cacheContext(); + // For MVCC case, let's enlist batch elements one by one. + if (plan.hasRows() && plan.mode() == UpdateMode.INSERT && !cctx.mvccEnabled()) { CacheOperationContext opCtx = setKeepBinaryContext(cctx); try { List>> cur = plan.createRows(argss); - List res = processDmlSelectResultBatched(plan, cur, fieldsQry.getPageSize()); - - return res; + return processDmlSelectResultBatched(plan, cur, fieldsQry.getPageSize()); } finally { cctx.operationContextPerCall(opCtx); @@ -344,8 +357,6 @@ List>> updateSqlFieldsDistributed(String schemaName, Con else { UpdateResult res = updateSqlFields(schemaName, c, p, fieldsQry, false, null, cancel); - ArrayList>> resCurs = new ArrayList<>(1); - checkUpdateResult(res); QueryCursorImpl> resCur = (QueryCursorImpl>)new QueryCursorImpl(Collections.singletonList @@ -353,9 +364,7 @@ List>> updateSqlFieldsDistributed(String schemaName, Con resCur.fieldsMeta(UPDATE_RESULT_META); - resCurs.add(resCur); - - return resCurs; + return Collections.singletonList(resCur); } } @@ -419,7 +428,7 @@ long streamUpdateQuery(String schemaName, IgniteDataStreamer streamer, PreparedS if (!F.isEmpty(plan.selectQuery())) { GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()), plan.selectQuery(), F.asList(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)), - null, false, 0, null); + null, false, false, 0, null); it = res.iterator(); } @@ -467,9 +476,7 @@ long streamUpdateQuery(String schemaName, IgniteDataStreamer streamer, PreparedS * Actually perform SQL DML operation locally. * * @param schemaName Schema name. - * @param cctx Cache context. - * @param c Connection. - * @param prepared Prepared statement for DML query. + * @param plan Cache context. * @param fieldsQry Fields query. * @param loc Local query flag. * @param filters Cache name and key filter. @@ -478,12 +485,124 @@ long streamUpdateQuery(String schemaName, IgniteDataStreamer streamer, PreparedS * @throws IgniteCheckedException if failed. */ @SuppressWarnings({"ConstantConditions", "unchecked"}) - private UpdateResult executeUpdateStatement(String schemaName, final GridCacheContext cctx, Connection c, - Prepared prepared, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters, + private UpdateResult executeUpdateStatement(String schemaName, final UpdatePlan plan, + SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException { - Integer errKeysPos = null; + GridCacheContext cctx = plan.cacheContext(); + + if (cctx != null && cctx.mvccEnabled()) { + assert cctx.transactional(); + + DmlDistributedPlanInfo distributedPlan = plan.distributedPlan(); + + GridNearTxLocal tx = tx(cctx.kernalContext()); + + boolean implicit = (tx == null); + + boolean commit = implicit && (!(fieldsQry instanceof SqlFieldsQueryEx) || + ((SqlFieldsQueryEx)fieldsQry).isAutoCommit()); + + if (implicit) + tx = txStart(cctx, fieldsQry.getTimeout()); + + requestSnapshot(cctx, checkActive(tx)); + + try (GridNearTxLocal toCommit = commit ? tx : null) { + long timeout; + + if (implicit) + timeout = tx.remainingTime(); + else { + long tm1 = tx.remainingTime(), tm2 = fieldsQry.getTimeout(); + + timeout = tm1 > 0 && tm2 > 0 ? Math.min(tm1, tm2) : Math.max(tm1, tm2); + } + + if (cctx.isReplicated() || distributedPlan == null || ((plan.mode() == UpdateMode.INSERT + || plan.mode() == UpdateMode.MERGE) && !plan.isLocalSubquery())) { + + boolean sequential = true; + + UpdateSourceIterator it; + + if (plan.fastResult()) { + IgniteBiTuple row = plan.getFastRow(fieldsQry.getArgs()); + + EnlistOperation op = UpdatePlan.enlistOperation(plan.mode()); + + it = new DmlUpdateSingleEntryIterator<>(op, op.isDeleteOrLock() ? row.getKey() : row); + } + else if (plan.hasRows()) + it = new DmlUpdateResultsIterator(UpdatePlan.enlistOperation(plan.mode()), plan, plan.createRows(fieldsQry.getArgs())); + else { + // TODO IGNITE-8865 if there is no ORDER BY statement it's no use to retain entries order on locking (sequential = false). + SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQuery(), fieldsQry.isCollocated()) + .setArgs(fieldsQry.getArgs()) + .setDistributedJoins(fieldsQry.isDistributedJoins()) + .setEnforceJoinOrder(fieldsQry.isEnforceJoinOrder()) + .setLocal(fieldsQry.isLocal()) + .setPageSize(fieldsQry.getPageSize()) + .setTimeout((int)timeout, TimeUnit.MILLISECONDS); + + FieldsQueryCursor> cur = idx.querySqlFields(schemaName, newFieldsQry, null, + true, true, mvccTracker(cctx, tx), cancel).get(0); + + it = plan.iteratorForTransaction(idx, cur); + } + + IgniteInternalFuture fut = tx.updateAsync(cctx, it, + fieldsQry.getPageSize(), timeout, sequential); + + UpdateResult res = new UpdateResult(fut.get(), X.EMPTY_OBJECT_ARRAY); + + if (commit) + toCommit.commit(); + + return res; + } + + int[] ids = U.toIntArray(distributedPlan.getCacheIds()); + + int flags = 0; + + if (fieldsQry.isEnforceJoinOrder()) + flags |= GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER; + + if (distributedPlan.isReplicatedOnly()) + flags |= GridH2QueryRequest.FLAG_REPLICATED; + + int[] parts = fieldsQry.getPartitions(); + + IgniteInternalFuture fut = tx.updateAsync( + cctx, + ids, + parts, + schemaName, + fieldsQry.getSql(), + fieldsQry.getArgs(), + flags, + fieldsQry.getPageSize(), + timeout); - UpdatePlan plan = getPlanForStatement(schemaName, c, prepared, fieldsQry, loc, errKeysPos); + UpdateResult res = new UpdateResult(fut.get(), X.EMPTY_OBJECT_ARRAY); + + if (commit) + toCommit.commit(); + + return res; + } + catch (IgniteCheckedException e) { + checkSqlException(e); + + U.error(log, "Error during update [localNodeId=" + cctx.localNodeId() + "]", e); + + throw new IgniteSQLException("Failed to run update. " + e.getMessage(), e); + } + finally { + if (commit) + cctx.tm().resetContext(); + } + } UpdateResult fastUpdateRes = plan.processFast(fieldsQry.getArgs()); @@ -514,13 +633,14 @@ private UpdateResult executeUpdateStatement(String schemaName, final GridCacheCo .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS); cur = (QueryCursorImpl>)idx.querySqlFields(schemaName, newFieldsQry, null, true, true, - cancel).get(0); + null, cancel).get(0); } else if (plan.hasRows()) cur = plan.createRows(fieldsQry.getArgs()); else { final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQuery(), - F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel); + F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), false, fieldsQry.getTimeout(), + cancel); cur = new QueryCursorImpl<>(new Iterable>() { @Override public Iterator> iterator() { @@ -536,7 +656,17 @@ else if (plan.hasRows()) int pageSize = loc ? 0 : fieldsQry.getPageSize(); - return processDmlSelectResult(cctx, plan, cur, pageSize); + return processDmlSelectResult(plan, cur, pageSize); + } + + /** + * @param e Exception. + */ + private void checkSqlException(IgniteCheckedException e) { + IgniteSQLException sqlEx = X.cause(e, IgniteSQLException.class); + + if(sqlEx != null) + throw sqlEx; } /** @@ -564,14 +694,13 @@ private List processDmlSelectResultBatched(UpdatePlan plan, List> cursor, + private UpdateResult processDmlSelectResult(UpdatePlan plan, Iterable> cursor, int pageSize) throws IgniteCheckedException { switch (plan.mode()) { case MERGE: @@ -584,7 +713,7 @@ private UpdateResult processDmlSelectResult(GridCacheContext cctx, UpdatePlan pl return doUpdate(plan, cursor, pageSize); case DELETE: - return doDelete(cctx, cursor, pageSize); + return doDelete(plan.cacheContext(), cursor, pageSize); default: throw new IgniteSQLException("Unexpected DML operation [mode=" + plan.mode() + ']', @@ -606,6 +735,8 @@ private UpdateResult processDmlSelectResult(GridCacheContext cctx, UpdatePlan pl @SuppressWarnings({"unchecked", "ConstantConditions"}) UpdatePlan getPlanForStatement(String schema, Connection conn, Prepared p, SqlFieldsQuery fieldsQry, boolean loc, @Nullable Integer errKeysPos) throws IgniteCheckedException { + isDmlOnSchemaSupported(schema); + H2CachedStatementKey planKey = H2CachedStatementKey.forDmlStatement(schema, p.getSQL(), fieldsQry, loc); UpdatePlan res = (errKeysPos == null ? planCache.get(planKey) : null); @@ -972,6 +1103,82 @@ UpdateResult mapDistributedUpdate(String schemaName, PreparedStatement stmt, Sql return updateSqlFields(schemaName, c, GridSqlQueryParser.prepared(stmt), fldsQry, local, filter, cancel); } + /** + * @param schema Schema name. + * @param conn Connection. + * @param stmt Prepared statement. + * @param qry Sql fields query + * @param filter Backup filter. + * @param cancel Query cancel object. + * @param local {@code true} if should be executed locally. + * @param topVer Topology version. + * @param mvccSnapshot MVCC snapshot. + * @return Iterator upon updated values. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("unchecked") + public UpdateSourceIterator prepareDistributedUpdate(String schema, Connection conn, + PreparedStatement stmt, SqlFieldsQuery qry, + IndexingQueryFilter filter, GridQueryCancel cancel, boolean local, + AffinityTopologyVersion topVer, MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + + Prepared prepared = GridSqlQueryParser.prepared(stmt); + + UpdatePlan plan = getPlanForStatement(schema, conn, prepared, qry, local, null); + + GridCacheContext cctx = plan.cacheContext(); + + CacheOperationContext opCtx = cctx.operationContextPerCall(); + + // Force keepBinary for operation context to avoid binary deserialization inside entry processor + if (cctx.binaryMarshaller()) { + CacheOperationContext newOpCtx = null; + + if (opCtx == null) + newOpCtx = new CacheOperationContext().keepBinary(); + else if (!opCtx.isKeepBinary()) + newOpCtx = opCtx.keepBinary(); + + if (newOpCtx != null) + cctx.operationContextPerCall(newOpCtx); + } + + QueryCursorImpl> cur; + + // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual + // sub-query and not some dummy stuff like "select 1, 2, 3;" + if (!local && !plan.isLocalSubquery()) { + SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQuery(), qry.isCollocated()) + .setArgs(qry.getArgs()) + .setDistributedJoins(qry.isDistributedJoins()) + .setEnforceJoinOrder(qry.isEnforceJoinOrder()) + .setLocal(qry.isLocal()) + .setPageSize(qry.getPageSize()) + .setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS); + + cur = (QueryCursorImpl>)idx.querySqlFields(schema, newFieldsQry, null, true, true, + new StaticMvccQueryTracker(cctx, mvccSnapshot), cancel).get(0); + } + else { + final GridQueryFieldsResult res = idx.queryLocalSqlFields(schema, plan.selectQuery(), + F.asList(qry.getArgs()), filter, qry.isEnforceJoinOrder(), false, qry.getTimeout(), cancel, + new StaticMvccQueryTracker(cctx, mvccSnapshot)); + + cur = new QueryCursorImpl<>(new Iterable>() { + @Override public Iterator> iterator() { + try { + return res.iterator(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + }, cancel); + } + + return plan.iteratorForTransaction(idx, cur); + } + /** * Runs a DML statement for which we have internal command executor. * @@ -1041,7 +1248,7 @@ public FieldsQueryCursor> processBulkLoadCommand(SqlBulkLoadCommand cmd) } /** */ - private final static class InsertEntryProcessor implements EntryProcessor { + private static final class InsertEntryProcessor implements EntryProcessor { /** Value to set. */ private final Object val; @@ -1064,7 +1271,7 @@ private InsertEntryProcessor(Object val) { /** * Entry processor invoked by UPDATE and DELETE operations. */ - private final static class ModifyingEntryProcessor implements EntryProcessor { + private static final class ModifyingEntryProcessor implements EntryProcessor { /** Value to expect. */ private final Object val; @@ -1138,6 +1345,17 @@ static boolean isDmlStatement(Prepared stmt) { return stmt instanceof Merge || stmt instanceof Insert || stmt instanceof Update || stmt instanceof Delete; } + /** + * Check if schema supports DDL statement. + * + * @param schemaName Schema name. + */ + private static void isDmlOnSchemaSupported(String schemaName) { + if (F.eq(QueryUtils.SCHEMA_SYS, schemaName)) + throw new IgniteSQLException("DML statements are not supported on " + schemaName + " schema", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + } + /** * Check update result for erroneous keys and throws concurrent update exception if necessary. * @@ -1181,4 +1399,82 @@ private BulkLoadDataConverter(UpdatePlan plan) { return plan.processRow(record); } } + + /** */ + private static class DmlUpdateResultsIterator + implements UpdateSourceIterator { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private EnlistOperation op; + + /** */ + private UpdatePlan plan; + + /** */ + private Iterator> it; + + /** */ + DmlUpdateResultsIterator(EnlistOperation op, UpdatePlan plan, Iterable> rows) { + this.op = op; + this.plan = plan; + this.it = rows.iterator(); + } + + /** {@inheritDoc} */ + @Override public EnlistOperation operation() { + return op; + } + + /** {@inheritDoc} */ + public boolean hasNextX() { + return it.hasNext(); + } + + /** {@inheritDoc} */ + public Object nextX() throws IgniteCheckedException { + return plan.processRowForTx(it.next()); + } + } + + /** */ + private static class DmlUpdateSingleEntryIterator implements UpdateSourceIterator { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private EnlistOperation op; + + /** */ + private boolean first = true; + + /** */ + private T entry; + + /** */ + DmlUpdateSingleEntryIterator(EnlistOperation op, T entry) { + this.op = op; + this.entry = entry; + } + + /** {@inheritDoc} */ + @Override public EnlistOperation operation() { + return op; + } + + /** {@inheritDoc} */ + public boolean hasNextX() { + return first; + } + + /** {@inheritDoc} */ + public T nextX() { + T res = first ? entry : null; + + first = false; + + return res; + } + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java index e180c9c8d2d4c..425015a574ee1 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ConnectionWrapper.java @@ -17,26 +17,34 @@ package org.apache.ignite.internal.processors.query.h2; +import java.sql.Connection; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; -import java.sql.Connection; - /** - * Wrapper to store connection and flag is schema set or not. + * Wrapper to store connection with currently used schema and statement cache. */ -public class H2ConnectionWrapper { +public class H2ConnectionWrapper implements AutoCloseable { /** */ - private Connection conn; + private static final int STATEMENT_CACHE_SIZE = 256; + + /** */ + private final Connection conn; /** */ private volatile String schema; + /** */ + private volatile H2StatementCache statementCache; + /** * @param conn Connection to use. */ H2ConnectionWrapper(Connection conn) { this.conn = conn; + + initStatementCache(); } /** @@ -60,8 +68,42 @@ public Connection connection() { return conn; } + /** + * @return Statement cache corresponding to connection. + */ + public H2StatementCache statementCache() { + return statementCache; + } + + /** + * Clears statement cache. + */ + public void clearStatementCache() { + initStatementCache(); + } + + /** + * @return Statement cache size. + */ + public int statementCacheSize() { + return statementCache == null ? 0 : statementCache.size(); + } + + /** + * Initializes statement cache. + */ + private void initStatementCache() { + statementCache = new H2StatementCache(STATEMENT_CACHE_SIZE); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(H2ConnectionWrapper.class, this); } + + /** Closes wrapped connection */ + @Override + public void close() { + U.closeQuiet(conn); + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java index 26cbaee63aa73..05df754394236 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java @@ -17,12 +17,12 @@ package org.apache.ignite.internal.processors.query.h2; -import org.apache.ignite.IgniteCheckedException; - import java.sql.ResultSet; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; /** * Special field set iterator based on database result set. @@ -31,20 +31,39 @@ public class H2FieldsIterator extends H2ResultSetIterator> { /** */ private static final long serialVersionUID = 0L; + /** */ + private transient MvccQueryTracker mvccTracker; + /** * @param data Data. + * @param mvccTracker Mvcc tracker. + * @param forUpdate {@code SELECT FOR UPDATE} flag. * @throws IgniteCheckedException If failed. */ - public H2FieldsIterator(ResultSet data) throws IgniteCheckedException { - super(data); + public H2FieldsIterator(ResultSet data, MvccQueryTracker mvccTracker, boolean forUpdate) + throws IgniteCheckedException { + super(data, forUpdate); + + this.mvccTracker = mvccTracker; } /** {@inheritDoc} */ @Override protected List createRow() { - ArrayList res = new ArrayList<>(row.length); + List res = new ArrayList<>(row.length); Collections.addAll(res, row); return res; } + + /** {@inheritDoc} */ + @Override public void onClose() { + try { + super.onClose(); + } + finally { + if (mvccTracker != null) + mvccTracker.onDone(); + } + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java index 31add346ee01f..770d9d5eb8d72 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2KeyValueIterator.java @@ -34,7 +34,7 @@ public class H2KeyValueIterator extends H2ResultSetIterator extends GridCloseableIteratorAdapte /** * @param data Data array. + * @param forUpdate Whether is result is one of {@code SELECT FOR UPDATE} query. * @throws IgniteCheckedException If failed. */ - protected H2ResultSetIterator(ResultSet data) throws IgniteCheckedException { + protected H2ResultSetIterator(ResultSet data, boolean forUpdate) throws IgniteCheckedException { this.data = data; try { @@ -83,7 +84,9 @@ protected H2ResultSetIterator(ResultSet data) throws IgniteCheckedException { if (data != null) { try { - row = new Object[data.getMetaData().getColumnCount()]; + int colsCnt = data.getMetaData().getColumnCount(); + + row = new Object[forUpdate ? colsCnt - 1 : colsCnt]; } catch (SQLException e) { throw new IgniteCheckedException(e); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java index 673625f865d53..6426994063cd7 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2StatementCache.java @@ -17,52 +17,59 @@ package org.apache.ignite.internal.processors.query.h2; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + import java.sql.PreparedStatement; import java.util.LinkedHashMap; import java.util.Map; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.Nullable; /** - * Statement cache. + * Statement cache. LRU eviction policy is used. Not thread-safe. */ -class H2StatementCache extends LinkedHashMap { - /** */ - private int size; - +final class H2StatementCache { /** Last usage. */ private volatile long lastUsage; + /** */ + private final LinkedHashMap lruStmtCache; + /** - * @param size Size. + * @param size Maximum number of statements this cache can store. */ H2StatementCache(int size) { - super(size, (float)0.75, true); - - this.size = size; - } - - /** {@inheritDoc} */ - @Override protected boolean removeEldestEntry(Map.Entry eldest) { - boolean rmv = size() > size; + lruStmtCache = new LinkedHashMap(size, .75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + if (size() <= size) + return false; - if (rmv) { - PreparedStatement stmt = eldest.getValue(); + U.closeQuiet(eldest.getValue()); - U.closeQuiet(stmt); - } + return true; + } + }; + } - return rmv; + /** + * Caches a statement. + * + * @param key Key associated with statement. + * @param stmt Statement which will be cached. + */ + void put(H2CachedStatementKey key, @NotNull PreparedStatement stmt) { + lruStmtCache.put(key, stmt); } /** - * Get statement for given schema and SQL. - * @param schemaName Schema name. - * @param sql SQL statement. - * @return Cached {@link PreparedStatement}, or {@code null} if none found. + * Retrieves cached statement. + * + * @param key Key for a statement. + * @return Statement associated with a key. */ - @Nullable public PreparedStatement get(String schemaName, String sql) { - return get(new H2CachedStatementKey(schemaName, sql)); + @Nullable PreparedStatement get(H2CachedStatementKey key) { + return lruStmtCache.get(key); } /** @@ -70,24 +77,31 @@ class H2StatementCache extends LinkedHashMap UPDATE_RESULT_META = Collections. singletonList(new H2SqlFieldMetadata(null, null, "UPDATED", Long.class.getName(), -1, -1)); - /** */ - private static final int PREPARED_STMT_CACHE_SIZE = 256; - /** */ private static final int TWO_STEP_QRY_CACHE_SIZE = 1024; - /** The period of clean up the {@link #stmtCache}. */ + /** The period of clean up the statement cache. */ private final Long CLEANUP_STMT_CACHE_PERIOD = Long.getLong(IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD, 10_000); /** The period of clean up the {@link #conns}. */ @SuppressWarnings("FieldCanBeLocal") private final Long CLEANUP_CONNECTIONS_PERIOD = 2000L; - /** The timeout to remove entry from the {@link #stmtCache} if the thread doesn't perform any queries. */ + /** The timeout to remove entry from the statement cache if the thread doesn't perform any queries. */ private final Long STATEMENT_CACHE_THREAD_USAGE_TIMEOUT = Long.getLong(IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT, 600 * 1000); @@ -274,7 +295,8 @@ public class IgniteH2Indexing implements GridQueryIndexing { private String dbUrl = "jdbc:h2:mem:"; /** */ - private final ConcurrentMap conns = new ConcurrentHashMap<>(); + // TODO https://issues.apache.org/jira/browse/IGNITE-9062 + private final ConcurrentMap conns = new ConcurrentHashMap<>(); /** */ private GridMapQueryExecutor mapQryExec; @@ -301,44 +323,39 @@ public class IgniteH2Indexing implements GridQueryIndexing { private final H2RowCacheRegistry rowCache = new H2RowCacheRegistry(); /** */ - private final ThreadLocal connCache = new ThreadLocal() { - @Override public H2ConnectionWrapper get() { - H2ConnectionWrapper c = super.get(); + // TODO https://issues.apache.org/jira/browse/IGNITE-9062 + private final ThreadLocalObjectPool connectionPool = new ThreadLocalObjectPool<>(IgniteH2Indexing.this::newConnectionWrapper, 5); + + /** */ + // TODO https://issues.apache.org/jira/browse/IGNITE-9062 + private final ThreadLocal> connCache = new ThreadLocal>() { + @Override public ThreadLocalObjectPool.Reusable get() { + ThreadLocalObjectPool.Reusable reusable = super.get(); boolean reconnect = true; try { - reconnect = c == null || c.connection().isClosed(); + reconnect = reusable == null || reusable.object().connection().isClosed(); } catch (SQLException e) { U.warn(log, "Failed to check connection status.", e); } if (reconnect) { - c = initialValue(); + reusable = initialValue(); - set(c); - - // Reset statement cache when new connection is created. - stmtCache.remove(Thread.currentThread()); + set(reusable); } - return c; + return reusable; } - @Override protected H2ConnectionWrapper initialValue() { - Connection c; + @Override protected ThreadLocalObjectPool.Reusable initialValue() { + ThreadLocalObjectPool.Reusable reusableConnection = connectionPool.borrow(); - try { - c = DriverManager.getConnection(dbUrl); - } - catch (SQLException e) { - throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e); - } + conns.put(Thread.currentThread(), reusableConnection.object()); - conns.put(Thread.currentThread(), c); - - return new H2ConnectionWrapper(c); + return reusableConnection; } }; @@ -357,9 +374,6 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** */ private final ConcurrentMap dataTables = new ConcurrentHashMap<>(); - /** Statement cache. */ - private final ConcurrentHashMap stmtCache = new ConcurrentHashMap<>(); - /** */ private volatile GridBoundedConcurrentLinkedHashMap twoStepCache = new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE); @@ -419,6 +433,15 @@ private Connection systemConnection() { return sysConn; } + /** */ + private H2ConnectionWrapper newConnectionWrapper() { + try { + return new H2ConnectionWrapper(DriverManager.getConnection(dbUrl)); + } catch (SQLException e) { + throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e); + } + } + /** * @param c Connection. * @param sql SQL. @@ -443,7 +466,7 @@ private Connection systemConnection() { * @throws SQLException If failed. */ @SuppressWarnings("ConstantConditions") - @NotNull private PreparedStatement prepareStatement(Connection c, String sql, boolean useStmtCache) + @NotNull public PreparedStatement prepareStatement(Connection c, String sql, boolean useStmtCache) throws SQLException { return prepareStatement(c, sql, useStmtCache, false); } @@ -468,7 +491,7 @@ private Connection systemConnection() { PreparedStatement stmt = cache.get(key); - if (stmt != null && !stmt.isClosed() && !((JdbcStatement)stmt).isCancelled() && + if (stmt != null && !stmt.isClosed() && !stmt.unwrap(JdbcStatement.class).isCancelled() && !GridSqlQueryParser.prepared(stmt).needRecompile()) { assert stmt.getConnection() == c; @@ -478,9 +501,7 @@ private Connection systemConnection() { if (cachedOnly) return null; - stmt = prepare0(c, sql); - - cache.put(key, stmt); + cache.put(key, stmt = PreparedStatementExImpl.wrap(prepare0(c, sql))); return stmt; } @@ -503,36 +524,25 @@ private PreparedStatement prepare0(Connection c, String sql) throws SQLException GridH2Table.insertHack(true); try { - return c.prepareStatement(sql); + return c.prepareStatement(sql, ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY); } finally { GridH2Table.insertHack(false); } } else - return c.prepareStatement(sql); + return c.prepareStatement(sql, ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY); } /** * @return {@link H2StatementCache} associated with current thread. */ @NotNull private H2StatementCache getStatementsCacheForCurrentThread() { - Thread curThread = Thread.currentThread(); - - H2StatementCache cache = stmtCache.get(curThread); - - if (cache == null) { - H2StatementCache cache0 = new H2StatementCache(PREPARED_STMT_CACHE_SIZE); - - cache = stmtCache.putIfAbsent(curThread, cache0); + H2StatementCache statementCache = connCache.get().object().statementCache(); - if (cache == null) - cache = cache0; - } - - cache.updateLastUsage(); + statementCache.updateLastUsage(); - return cache; + return statementCache; } /** {@inheritDoc} */ @@ -550,7 +560,7 @@ private PreparedStatement prepare0(Connection c, String sql) throws SQLException * @throws IgniteCheckedException In case of error. */ private Connection connectionForThread(@Nullable String schema) throws IgniteCheckedException { - H2ConnectionWrapper c = connCache.get(); + H2ConnectionWrapper c = connCache.get().object(); if (c == null) throw new IgniteCheckedException("Failed to get DB connection for thread (check log for details)."); @@ -724,7 +734,7 @@ else if (obj instanceof BigDecimal) * Handles SQL exception. */ private void onSqlException() { - Connection conn = connCache.get().connection(); + Connection conn = connCache.get().object().connection(); connCache.set(null); @@ -737,8 +747,12 @@ private void onSqlException() { } /** {@inheritDoc} */ - @Override public void store(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row, - @Nullable CacheDataRow prevRow, boolean prevRowAvailable) throws IgniteCheckedException { + @Override public void store(GridCacheContext cctx, + GridQueryTypeDescriptor type, + CacheDataRow row, + @Nullable CacheDataRow prevRow, + boolean prevRowAvailable) throws IgniteCheckedException + { String cacheName = cctx.name(); H2TableDescriptor tbl = tableDescriptor(schema(cacheName), cacheName, type.name()); @@ -746,7 +760,7 @@ private void onSqlException() { if (tbl == null) return; // Type was rejected. - tbl.table().update(row, prevRow, prevRowAvailable); + tbl.table().update(row, prevRow, prevRowAvailable); if (tbl.luceneIndex() != null) { long expireTime = row.expireTime(); @@ -1024,76 +1038,230 @@ GridH2IndexBase createSortedIndex(String name, GridH2Table tbl, boolean pk, List * @param params Query parameters. * @param filter Cache name and key filter. * @param enforceJoinOrder Enforce join order of tables in the query. + * @param startTx Start transaction flag. * @param timeout Query timeout in milliseconds. * @param cancel Query cancel. * @return Query result. * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") - GridQueryFieldsResult queryLocalSqlFields(final String schemaName, final String qry, + public GridQueryFieldsResult queryLocalSqlFields(String schemaName, String qry, @Nullable Collection params, + IndexingQueryFilter filter, boolean enforceJoinOrder, boolean startTx, int timeout, + GridQueryCancel cancel) throws IgniteCheckedException { + return queryLocalSqlFields(schemaName, qry, params, filter, enforceJoinOrder, startTx, timeout, cancel, null); + } + + /** + * Queries individual fields (generally used by JDBC drivers). + * + * @param schemaName Schema name. + * @param qry Query. + * @param params Query parameters. + * @param filter Cache name and key filter. + * @param enforceJoinOrder Enforce join order of tables in the query. + * @param startTx Start transaction flag. + * @param timeout Query timeout in milliseconds. + * @param cancel Query cancel. + * @param mvccTracker Query tracker. + * @return Query result. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("unchecked") + GridQueryFieldsResult queryLocalSqlFields(final String schemaName, String qry, @Nullable final Collection params, final IndexingQueryFilter filter, boolean enforceJoinOrder, - final int timeout, final GridQueryCancel cancel) throws IgniteCheckedException { - final Connection conn = connectionForSchema(schemaName); + boolean startTx, int timeout, final GridQueryCancel cancel, + MvccQueryTracker mvccTracker) throws IgniteCheckedException { - H2Utils.setupConnection(conn, false, enforceJoinOrder); + GridNearTxLocal tx = null; boolean mvccEnabled = mvccEnabled(kernalContext()); - final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true); + assert mvccEnabled || mvccTracker == null; - if (GridSqlQueryParser.checkMultipleStatements(stmt)) - throw new IgniteSQLException("Multiple statements queries are not supported for local queries"); + try { + final Connection conn = connectionForSchema(schemaName); - Prepared p = GridSqlQueryParser.prepared(stmt); + H2Utils.setupConnection(conn, false, enforceJoinOrder); - if (DmlStatementsProcessor.isDmlStatement(p)) { - SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry); + PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true); - if (params != null) - fldsQry.setArgs(params.toArray()); + if (GridSqlQueryParser.checkMultipleStatements(stmt)) + throw new IgniteSQLException("Multiple statements queries are not supported for local queries"); - fldsQry.setEnforceJoinOrder(enforceJoinOrder); - fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS); + Prepared p = GridSqlQueryParser.prepared(stmt); - return dmlProc.updateSqlFieldsLocal(schemaName, conn, p, fldsQry, filter, cancel); - } - else if (DdlStatementsProcessor.isDdlStatement(p)) - throw new IgniteSQLException("DDL statements are supported for the whole cluster only", - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + if (DmlStatementsProcessor.isDmlStatement(p)) { + SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry); - List meta; + if (params != null) + fldsQry.setArgs(params.toArray()); - try { - meta = H2Utils.meta(stmt.getMetaData()); - } - catch (SQLException e) { - throw new IgniteCheckedException("Cannot prepare query metadata", e); - } + fldsQry.setEnforceJoinOrder(enforceJoinOrder); + fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS); - final GridH2QueryContext ctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL) - .filter(filter).distributedJoinMode(OFF); + return dmlProc.updateSqlFieldsLocal(schemaName, conn, p, fldsQry, filter, cancel); + } + else if (DdlStatementsProcessor.isDdlStatement(p)) { + throw new IgniteSQLException("DDL statements are supported for the whole cluster only.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + } - return new GridQueryFieldsResultAdapter(meta, null) { - @Override public GridCloseableIterator> iterator() throws IgniteCheckedException { - assert GridH2QueryContext.get() == null; + final GridH2QueryContext ctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL) + .filter(filter).distributedJoinMode(OFF); - GridH2QueryContext.set(ctx); + boolean forUpdate = GridSqlQueryParser.isForUpdateQuery(p); - GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL_FIELDS, - schemaName, U.currentTimeMillis(), cancel, true); + if (forUpdate && !mvccEnabled) + throw new IgniteSQLException("SELECT FOR UPDATE query requires transactional " + + "cache with MVCC enabled.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); - runs.putIfAbsent(run.id(), run); + GridNearTxSelectForUpdateFuture sfuFut = null; - try { - ResultSet rs = executeSqlQueryWithTimer(stmt, conn, qry, params, timeout, cancel); + if (mvccEnabled) { + if (mvccTracker == null) + mvccTracker = mvccTracker(stmt, startTx); + + if (mvccTracker != null) { + ctx.mvccSnapshot(mvccTracker.snapshot()); - return new H2FieldsIterator(rs); + if ((tx = checkActive(tx(this.ctx))) != null) { + int tm1 = (int)tx.remainingTime(), tm2 = timeout; + + timeout = tm1 > 0 && tm2 > 0 ? Math.min(tm1, tm2) : Math.max(tm1, tm2); + } } - finally { - GridH2QueryContext.clearThreadLocal(); - runs.remove(run.id()); + if (forUpdate) { + if (mvccTracker == null) + throw new IgniteSQLException("SELECT FOR UPDATE query requires transactional " + + "cache with MVCC enabled.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + GridSqlStatement stmt0 = new GridSqlQueryParser(false).parse(p); + + qry = GridSqlQueryParser.rewriteQueryForUpdateIfNeeded(stmt0, forUpdate = tx != null); + + stmt = preparedStatementWithParams(conn, qry, params, true); + + if (forUpdate) { + GridCacheContext cctx = mvccTracker.context(); + + try { + if (tx.topologyVersionSnapshot() == null) + new TxTopologyVersionFuture(tx, cctx).get(); + } + catch (Exception e) { + throw new IgniteSQLException("Failed to lock topology for SELECT FOR UPDATE query.", e); + } + + sfuFut = new GridNearTxSelectForUpdateFuture(cctx, tx, timeout); + + sfuFut.initLocal(); + } } } - }; + + List meta; + + try { + meta = H2Utils.meta(stmt.getMetaData()); + + if (forUpdate) { + assert meta.size() >= 1; + + meta = meta.subList(0, meta.size() - 1); + } + } + catch (SQLException e) { + throw new IgniteCheckedException("Cannot prepare query metadata", e); + } + + GridNearTxLocal tx0 = tx; + MvccQueryTracker mvccTracker0 = mvccTracker; + GridNearTxSelectForUpdateFuture sfuFut0 = sfuFut; + PreparedStatement stmt0 = stmt; + String qry0 = qry; + int timeout0 = timeout; + + return new GridQueryFieldsResultAdapter(meta, null) { + @Override public GridCloseableIterator> iterator() throws IgniteCheckedException { + assert GridH2QueryContext.get() == null; + + GridH2QueryContext.set(ctx); + + GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry0, + SQL_FIELDS, schemaName, U.currentTimeMillis(), cancel, true); + + runs.putIfAbsent(run.id(), run); + + try { + ResultSet rs = executeSqlQueryWithTimer(stmt0, conn, qry0, params, timeout0, cancel); + + if (sfuFut0 != null) { + assert tx0.mvccSnapshot() != null; + + ResultSetEnlistFuture enlistFut = ResultSetEnlistFuture.future( + IgniteH2Indexing.this.ctx.localNodeId(), + tx0.nearXidVersion(), + tx0.mvccSnapshot(), + tx0.threadId(), + IgniteUuid.randomUuid(), + -1, + null, + tx0, + timeout0, + sfuFut0.cache(), + rs + ); + + enlistFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + if (fut.error() != null) + sfuFut0.onResult(IgniteH2Indexing.this.ctx.localNodeId(), 0L, false, fut.error()); + else + sfuFut0.onResult(IgniteH2Indexing.this.ctx.localNodeId(), fut.result(), false, null); + } + }); + + enlistFut.init(); + + try { + sfuFut0.get(); + + rs.beforeFirst(); + } + catch (Exception e) { + U.closeQuiet(rs); + + throw new IgniteSQLException("Failed to obtain locks on result of SELECT FOR UPDATE.", + e); + } + } + + return new H2FieldsIterator(rs, mvccTracker0, sfuFut0 != null); + } + catch (IgniteCheckedException | RuntimeException | Error e) { + try { + if (mvccTracker0 != null) + mvccTracker0.onDone(); + } + catch (Exception e0) { + e.addSuppressed(e0); + } + + throw e; + } + finally { + GridH2QueryContext.clearThreadLocal(); + + runs.remove(run.id()); + } + } + }; + } + catch (IgniteCheckedException | RuntimeException | Error e) { + if (mvccEnabled && (tx != null || (tx = tx(ctx)) != null)) + tx.setRollbackOnly(); + + throw e; + } } /** {@inheritDoc} */ @@ -1171,7 +1339,7 @@ private static List zeroBatchedStreamedUpdateResult(int size) { * @return Prepared statement with set parameters. * @throws IgniteCheckedException If failed. */ - private PreparedStatement preparedStatementWithParams(Connection conn, String sql, Collection params, + public PreparedStatement preparedStatementWithParams(Connection conn, String sql, Collection params, boolean useStmtCache) throws IgniteCheckedException { final PreparedStatement stmt; @@ -1294,7 +1462,7 @@ public ResultSet executeSqlQueryWithTimer(Connection conn, String sql, @Nullable * @return Result. * @throws IgniteCheckedException If failed. */ - private ResultSet executeSqlQueryWithTimer(PreparedStatement stmt, Connection conn, String sql, + public ResultSet executeSqlQueryWithTimer(PreparedStatement stmt, Connection conn, String sql, @Nullable Collection params, int timeoutMillis, @Nullable GridQueryCancel cancel) throws IgniteCheckedException { long start = U.currentTimeMillis(); @@ -1350,10 +1518,12 @@ public void bindParameters(PreparedStatement stmt, @Override public FieldsQueryCursor> queryLocalSqlFields(String schemaName, SqlFieldsQuery qry, final boolean keepBinary, IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException { String sql = qry.getSql(); - Object[] args = qry.getArgs(); + List params = F.asList(qry.getArgs()); + boolean enforceJoinOrder = qry.isEnforceJoinOrder(), startTx = autoStartTx(qry); + int timeout = qry.getTimeout(); - final GridQueryFieldsResult res = queryLocalSqlFields(schemaName, sql, F.asList(args), filter, - qry.isEnforceJoinOrder(), qry.getTimeout(), cancel); + final GridQueryFieldsResult res = queryLocalSqlFields(schemaName, sql, params, filter, + enforceJoinOrder, startTx, timeout, cancel); QueryCursorImpl> cursor = new QueryCursorImpl<>(new Iterable>() { @SuppressWarnings("NullableProblems") @@ -1445,8 +1615,17 @@ GridCloseableIterator> queryLocalSql(String schemaNam H2Utils.setupConnection(conn, false, false); - GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter) - .distributedJoinMode(OFF)); + GridH2QueryContext qctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter) + .distributedJoinMode(OFF); + + PreparedStatement stmt = preparedStatementWithParams(conn, sql, params, true); + + MvccQueryTracker mvccTracker = mvccTracker(stmt, false); + + if (mvccTracker != null) + qctx.mvccSnapshot(mvccTracker.snapshot()); + + GridH2QueryContext.set(qctx); GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, schemaName, U.currentTimeMillis(), null, true); @@ -1454,27 +1633,88 @@ GridCloseableIterator> queryLocalSql(String schemaNam runs.put(run.id(), run); try { - ResultSet rs = executeSqlQueryWithTimer(conn, sql, params, true, 0, cancel); + ResultSet rs = executeSqlQueryWithTimer(stmt, conn, sql, params, 0, cancel); return new H2KeyValueIterator(rs); } finally { GridH2QueryContext.clearThreadLocal(); + if (mvccTracker != null) + mvccTracker.onDone(); + runs.remove(run.id()); } } + /** + * Initialises MVCC filter and returns MVCC query tracker if needed. + * @param stmt Prepared statement. + * @param startTx Start transaction flag. + * @return MVCC query tracker or {@code null} if MVCC is disabled for involved caches. + */ + private MvccQueryTracker mvccTracker(PreparedStatement stmt, boolean startTx) throws IgniteCheckedException { + List caches; + try { + if (stmt.isWrapperFor(PreparedStatementEx.class)) { + PreparedStatementEx stmtEx = stmt.unwrap(PreparedStatementEx.class); + + caches = stmtEx.meta(INVOLVED_CACHES); + + if (caches == null) + stmtEx.putMeta(INVOLVED_CACHES, caches = parseInvolvedCaches(stmt)); + } + else + caches = parseInvolvedCaches(stmt); + } catch (SQLException e) { + throw new IgniteSQLException(e); + } + + GridCacheContext firstCctx = null; + boolean mvccEnabled = false; + + for (GridCacheContext cctx : caches) { + if (firstCctx == null) { + firstCctx = cctx; + mvccEnabled = firstCctx.mvccEnabled(); + } + else if (mvccEnabled != cctx.mvccEnabled()) + throw new IllegalStateException("Using caches with different mvcc settings in same query is forbidden."); + } + + return mvccEnabled ? MvccUtils.mvccTracker(firstCctx, startTx) : null; + } + + /** */ + private static List parseInvolvedCaches(PreparedStatement stmt) { + GridSqlQueryParser parser = new GridSqlQueryParser(false); + + parser.parse(GridSqlQueryParser.prepared(stmt)); + + List involvedCaches = new ArrayList<>(); + + for (Object o : parser.objectsMap().values()) { + if (o instanceof GridSqlAlias) + o = GridSqlAlias.unwrap((GridSqlAst) o); + if (o instanceof GridSqlTable && ((GridSqlTable) o).dataTable() != null) + involvedCaches.add(((GridSqlTable) o).dataTable().cache()); + } + + return involvedCaches; + } + /** * @param schemaName Schema name. * @param qry Query. * @param keepCacheObj Flag to keep cache object. * @param enforceJoinOrder Enforce join order of tables. + * @param startTx Start transaction flag. * @param timeoutMillis Query timeout. * @param cancel Cancel object. * @param params Query parameters. * @param parts Partitions. * @param lazy Lazy query execution flag. + * @param mvccTracker Query tracker. * @return Iterable result. */ private Iterable> runQueryTwoStep( @@ -1482,19 +1722,33 @@ private Iterable> runQueryTwoStep( final GridCacheTwoStepQuery qry, final boolean keepCacheObj, final boolean enforceJoinOrder, + boolean startTx, final int timeoutMillis, final GridQueryCancel cancel, final Object[] params, final int[] parts, - final boolean lazy - ) { - return new Iterable>() { - @SuppressWarnings("NullableProblems") - @Override public Iterator> iterator() { - return rdcQryExec.query(schemaName, qry, keepCacheObj, enforceJoinOrder, timeoutMillis, cancel, params, - parts, lazy); - } - }; + final boolean lazy, + MvccQueryTracker mvccTracker) { + assert !qry.mvccEnabled() || !F.isEmpty(qry.cacheIds()); + + try { + final MvccQueryTracker tracker = mvccTracker == null && qry.mvccEnabled() ? + MvccUtils.mvccTracker(ctx.cache().context().cacheContext(qry.cacheIds().get(0)), startTx) : mvccTracker; + + if (qry.forUpdate()) + qry.forUpdate(checkActive(tx(ctx)) != null); + + return new Iterable>() { + @SuppressWarnings("NullableProblems") + @Override public Iterator> iterator() { + return rdcQryExec.query(schemaName, qry, keepCacheObj, enforceJoinOrder, timeoutMillis, + cancel, params, parts, lazy, tracker); + } + }; + } + catch (IgniteCheckedException e) { + throw new CacheException(e); + } } /** @@ -1551,7 +1805,7 @@ UpdateResult runDistributedUpdate( fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS); final QueryCursor> res = - querySqlFields(schemaName, fqry, null, keepBinary, true, null).get(0); + querySqlFields(schemaName, fqry, null, keepBinary, true, null, null).get(0); final Iterable> converted = new Iterable>() { @Override public Iterator> iterator() { @@ -1587,21 +1841,22 @@ UpdateResult runDistributedUpdate( * Try executing query using native facilities. * * @param schemaName Schema name. - * @param sql Query. + * @param qry Query. * @param cliCtx Client context, or {@code null} if not applicable. * @return Result or {@code null} if cannot parse/process this query. */ - private List>> tryQueryDistributedSqlFieldsNative(String schemaName, String sql, + @SuppressWarnings({"ConstantConditions", "StatementWithEmptyBody"}) + private List>> tryQueryDistributedSqlFieldsNative(String schemaName, SqlFieldsQuery qry, @Nullable SqlClientContext cliCtx) { // Heuristic check for fast return. - if (!INTERNAL_CMD_RE.matcher(sql.trim()).find()) + if (!INTERNAL_CMD_RE.matcher(qry.getSql().trim()).find()) return null; // Parse. SqlCommand cmd; try { - SqlParser parser = new SqlParser(schemaName, sql); + SqlParser parser = new SqlParser(schemaName, qry.getSql()); cmd = parser.nextCommand(); @@ -1609,16 +1864,16 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (parser.nextCommand() != null) return null; - // Currently supported commands are: - // CREATE/DROP INDEX - // COPY - // ALTER TABLE - // SET STREAMING - // CREATE/ALTER/DROP USER - if (!(cmd instanceof SqlCreateIndexCommand || cmd instanceof SqlDropIndexCommand - || cmd instanceof SqlAlterTableCommand || cmd instanceof SqlBulkLoadCommand + if (!(cmd instanceof SqlCreateIndexCommand + || cmd instanceof SqlDropIndexCommand + || cmd instanceof SqlBeginTransactionCommand + || cmd instanceof SqlCommitTransactionCommand + || cmd instanceof SqlRollbackTransactionCommand + || cmd instanceof SqlBulkLoadCommand + || cmd instanceof SqlAlterTableCommand || cmd instanceof SqlSetStreamingCommand - || cmd instanceof SqlCreateUserCommand || cmd instanceof SqlAlterUserCommand + || cmd instanceof SqlCreateUserCommand + || cmd instanceof SqlAlterUserCommand || cmd instanceof SqlDropUserCommand)) return null; } @@ -1628,7 +1883,7 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri catch (Exception e) { // Cannot parse, return. if (log.isDebugEnabled()) - log.debug("Failed to parse SQL with native parser [qry=" + sql + ", err=" + e + ']'); + log.debug("Failed to parse SQL with native parser [qry=" + qry.getSql() + ", err=" + e + ']'); if (!IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK)) return null; @@ -1638,36 +1893,40 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri if (e instanceof SqlParseException) code = ((SqlParseException)e).code(); - throw new IgniteSQLException("Failed to parse DDL statement: " + sql + ": " + e.getMessage(), + throw new IgniteSQLException("Failed to parse DDL statement: " + qry.getSql() + ": " + e.getMessage(), code, e); } // Execute. - if (cmd instanceof SqlBulkLoadCommand) { - FieldsQueryCursor> cursor = dmlProc.runNativeDmlStatement(sql, cmd); - - return Collections.singletonList(cursor); - } - else if (cmd instanceof SqlSetStreamingCommand) { - if (cliCtx == null) - throw new IgniteSQLException("SET STREAMING command can only be executed from JDBC or ODBC driver."); - - SqlSetStreamingCommand setCmd = (SqlSetStreamingCommand)cmd; - - boolean on = setCmd.isTurnOn(); - - if (on) - cliCtx.enableStreaming(setCmd.allowOverwrite(), setCmd.flushFrequency(), - setCmd.perNodeBufferSize(), setCmd.perNodeParallelOperations(), setCmd.isOrdered()); + try { + if (cmd instanceof SqlCreateIndexCommand + || cmd instanceof SqlDropIndexCommand + || cmd instanceof SqlAlterTableCommand + || cmd instanceof SqlCreateUserCommand + || cmd instanceof SqlAlterUserCommand + || cmd instanceof SqlDropUserCommand) + return Collections.singletonList(ddlProc.runDdlStatement(qry.getSql(), cmd)); + else if (cmd instanceof SqlBulkLoadCommand) + return Collections.singletonList(dmlProc.runNativeDmlStatement(qry.getSql(), cmd)); + else if (cmd instanceof SqlSetStreamingCommand) { + if (cliCtx == null) + throw new IgniteSQLException("SET STREAMING command can only be executed from JDBC or ODBC driver."); + + SqlSetStreamingCommand setCmd = (SqlSetStreamingCommand)cmd; + + if (setCmd.isTurnOn()) + cliCtx.enableStreaming(setCmd.allowOverwrite(), setCmd.flushFrequency(), + setCmd.perNodeBufferSize(), setCmd.perNodeParallelOperations(), setCmd.isOrdered()); + else + cliCtx.disableStreaming(); + } else - cliCtx.disableStreaming(); + processTxCommand(cmd, qry); return Collections.singletonList(H2Utils.zeroCursor()); } - else { - FieldsQueryCursor> cursor = ddlProc.runDdlStatement(sql, cmd); - - return Collections.singletonList(cursor); + catch (IgniteCheckedException e) { + throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + qry.getSql() + ']', e); } } @@ -1685,88 +1944,207 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { IgniteQueryErrorCode.STMT_TYPE_MISMATCH); } + /** + * Process transactional command. + * @param cmd Command. + * @param qry Query. + * @throws IgniteCheckedException if failed. + */ + private void processTxCommand(SqlCommand cmd, SqlFieldsQuery qry) throws IgniteCheckedException { + if (!mvccEnabled(ctx)) + throw new IgniteSQLException("MVCC must be enabled in order to invoke transactional operation: " + + qry.getSql(), IgniteQueryErrorCode.MVCC_DISABLED); + + NestedTxMode nestedTxMode = qry instanceof SqlFieldsQueryEx ? ((SqlFieldsQueryEx)qry).getNestedTxMode() : + NestedTxMode.DEFAULT; + + GridNearTxLocal tx = tx(ctx); + + if (cmd instanceof SqlBeginTransactionCommand) { + if (tx != null) { + if (nestedTxMode == null) + nestedTxMode = NestedTxMode.DEFAULT; + + switch (nestedTxMode) { + case COMMIT: + doCommit(tx); + + txStart(ctx, qry.getTimeout()); + + break; + + case IGNORE: + log.warning("Transaction has already been started, ignoring BEGIN command."); + + break; + + case ERROR: + throw new IgniteSQLException("Transaction has already been started.", + IgniteQueryErrorCode.TRANSACTION_EXISTS); + + default: + throw new IgniteSQLException("Unexpected nested transaction handling mode: " + + nestedTxMode.name()); + } + } + else + txStart(ctx, qry.getTimeout()); + } + else if (cmd instanceof SqlCommitTransactionCommand) { + // Do nothing if there's no transaction. + if (tx != null) + doCommit(tx); + } + else { + assert cmd instanceof SqlRollbackTransactionCommand; + + // Do nothing if there's no transaction. + if (tx != null) + doRollback(tx); + } + } + + /** + * Commit and properly close transaction. + * @param tx Transaction. + * @throws IgniteCheckedException if failed. + */ + @SuppressWarnings("ThrowFromFinallyBlock") + private void doCommit(@NotNull GridNearTxLocal tx) throws IgniteCheckedException { + try { + if (!tx.isRollbackOnly()) + tx.commit(); + } + finally { + closeTx(tx); + } + } + + /** + * Rollback and properly close transaction. + * @param tx Transaction. + * @throws IgniteCheckedException if failed. + */ + @SuppressWarnings("ThrowFromFinallyBlock") + private void doRollback(@NotNull GridNearTxLocal tx) throws IgniteCheckedException { + try { + tx.rollback(); + } + finally { + closeTx(tx); + } + } + + /** + * Properly close transaction. + * @param tx Transaction. + * @throws IgniteCheckedException if failed. + */ + private void closeTx(@NotNull GridNearTxLocal tx) throws IgniteCheckedException { + try { + tx.close(); + } + finally { + ctx.cache().context().tm().resetContext(); + } + } /** {@inheritDoc} */ @SuppressWarnings({"StringEquality", "unchecked"}) @Override public List>> querySqlFields(String schemaName, SqlFieldsQuery qry, - @Nullable SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, GridQueryCancel cancel) { - List>> res = tryQueryDistributedSqlFieldsNative(schemaName, qry.getSql(), cliCtx); + @Nullable SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts, MvccQueryTracker tracker, + GridQueryCancel cancel) { + boolean mvccEnabled = mvccEnabled(ctx), startTx = autoStartTx(qry); - if (res != null) - return res; + try { + List>> res = tryQueryDistributedSqlFieldsNative(schemaName, qry, cliCtx); - { - // First, let's check if we already have a two-step query for this statement... - H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, qry.getSql(), - qry.isCollocated(), qry.isDistributedJoins(), qry.isEnforceJoinOrder(), qry.isLocal()); + if (res != null) + return res; - H2TwoStepCachedQuery cachedQry; + { + // First, let's check if we already have a two-step query for this statement... + H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, qry.getSql(), + qry.isCollocated(), qry.isDistributedJoins(), qry.isEnforceJoinOrder(), qry.isLocal()); - if ((cachedQry = twoStepCache.get(cachedQryKey)) != null) { - checkQueryType(qry, true); + H2TwoStepCachedQuery cachedQry; - GridCacheTwoStepQuery twoStepQry = cachedQry.query().copy(); + if ((cachedQry = twoStepCache.get(cachedQryKey)) != null) { + checkQueryType(qry, true); - List meta = cachedQry.meta(); + GridCacheTwoStepQuery twoStepQry = cachedQry.query().copy(); - res = Collections.singletonList(doRunDistributedQuery(schemaName, qry, twoStepQry, meta, keepBinary, - cancel)); + List meta = cachedQry.meta(); - if (!twoStepQry.explain()) - twoStepCache.putIfAbsent(cachedQryKey, new H2TwoStepCachedQuery(meta, twoStepQry.copy())); + res = Collections.singletonList(doRunDistributedQuery(schemaName, qry, twoStepQry, meta, keepBinary, + startTx, tracker, cancel)); - return res; + if (!twoStepQry.explain()) + twoStepCache.putIfAbsent(cachedQryKey, new H2TwoStepCachedQuery(meta, twoStepQry.copy())); + + return res; + } } - } - { - // Second, let's check if we already have a parsed statement... - PreparedStatement cachedStmt; + { + // Second, let's check if we already have a parsed statement... + PreparedStatement cachedStmt; - if ((cachedStmt = cachedStatement(connectionForSchema(schemaName), qry.getSql())) != null) { - Prepared prepared = GridSqlQueryParser.prepared(cachedStmt); + if ((cachedStmt = cachedStatement(connectionForSchema(schemaName), qry.getSql())) != null) { + Prepared prepared = GridSqlQueryParser.prepared(cachedStmt); - // We may use this cached statement only for local queries and non queries. - if (qry.isLocal() || !prepared.isQuery()) - return (List>>)doRunPrepared(schemaName, prepared, qry, null, - null, keepBinary, cancel); + // We may use this cached statement only for local queries and non queries. + if (qry.isLocal() || !prepared.isQuery()) + return (List>>)doRunPrepared(schemaName, prepared, qry, null, null, + keepBinary, startTx, tracker, cancel); + } } - } - res = new ArrayList<>(1); + res = new ArrayList<>(1); - int firstArg = 0; + int firstArg = 0; - String remainingSql = qry.getSql(); + String remainingSql = qry.getSql(); - while (remainingSql != null) { - ParsingResult parseRes = parseAndSplit(schemaName, - remainingSql != qry.getSql() ? cloneFieldsQuery(qry).setSql(remainingSql) : qry, firstArg); + while (remainingSql != null) { + ParsingResult parseRes = parseAndSplit(schemaName, + remainingSql != qry.getSql() ? cloneFieldsQuery(qry).setSql(remainingSql) : qry, firstArg); - // Let's avoid second reflection getter call by returning Prepared object too - Prepared prepared = parseRes.prepared(); + // Let's avoid second reflection getter call by returning Prepared object too + Prepared prepared = parseRes.prepared(); - GridCacheTwoStepQuery twoStepQry = parseRes.twoStepQuery(); + GridCacheTwoStepQuery twoStepQry = parseRes.twoStepQuery(); - List meta = parseRes.meta(); + List meta = parseRes.meta(); - SqlFieldsQuery newQry = parseRes.newQuery(); + SqlFieldsQuery newQry = parseRes.newQuery(); - remainingSql = parseRes.remainingSql(); + remainingSql = parseRes.remainingSql(); - if (remainingSql != null && failOnMultipleStmts) - throw new IgniteSQLException("Multiple statements queries are not supported"); + if (remainingSql != null && failOnMultipleStmts) + throw new IgniteSQLException("Multiple statements queries are not supported"); - firstArg += prepared.getParameters().size(); + firstArg += prepared.getParameters().size(); - res.addAll(doRunPrepared(schemaName, prepared, newQry, twoStepQry, meta, keepBinary, cancel)); + res.addAll(doRunPrepared(schemaName, prepared, newQry, twoStepQry, meta, keepBinary, startTx, tracker, + cancel)); - if (parseRes.twoStepQuery() != null && parseRes.twoStepQueryKey() != null && + if (parseRes.twoStepQuery() != null && parseRes.twoStepQueryKey() != null && !parseRes.twoStepQuery().explain()) - twoStepCache.putIfAbsent(parseRes.twoStepQueryKey(), new H2TwoStepCachedQuery(meta, twoStepQry.copy())); + twoStepCache.putIfAbsent(parseRes.twoStepQueryKey(), new H2TwoStepCachedQuery(meta, + twoStepQry.copy())); + } + + return res; } + catch (RuntimeException | Error e) { + GridNearTxLocal tx; - return res; + if (mvccEnabled && (tx = tx(ctx)) != null) + tx.setRollbackOnly(); + + throw e; + } } /** @@ -1777,12 +2155,15 @@ private void checkQueryType(SqlFieldsQuery qry, boolean isQry) { * @param twoStepQry Two-step query if this query must be executed in a distributed way. * @param meta Metadata for {@code twoStepQry}. * @param keepBinary Whether binary objects must not be deserialized automatically. - * @param cancel Query cancel state holder. @return Query result. + * @param startTx Start transactionq flag. + * @param tracker MVCC tracker. + * @param cancel Query cancel state holder. + * @return Query result. */ @SuppressWarnings("unchecked") private List>> doRunPrepared(String schemaName, Prepared prepared, - SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, - List meta, boolean keepBinary, GridQueryCancel cancel) { + SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, List meta, boolean keepBinary, + boolean startTx, MvccQueryTracker tracker, GridQueryCancel cancel) { String sqlQry = qry.getSql(); boolean loc = qry.isLocal(); @@ -1848,7 +2229,7 @@ private List>> doRunPrepared(String schemaNa checkQueryType(qry, true); return Collections.singletonList(doRunDistributedQuery(schemaName, qry, twoStepQry, meta, keepBinary, - cancel)); + startTx, tracker, cancel)); } // We've encountered a local query, let's just run it. @@ -1885,7 +2266,8 @@ private ParsingResult parseAndSplit(String schemaName, SqlFieldsQuery qry, int f PreparedStatement stmt = prepareStatementAndCaches(c, qry.getSql()); if (loc && GridSqlQueryParser.checkMultipleStatements(stmt)) - throw new IgniteSQLException("Multiple statements queries are not supported for local queries"); + throw new IgniteSQLException("Multiple statements queries are not supported for local queries.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); GridSqlQueryParser.PreparedWithRemaining prep = GridSqlQueryParser.preparedWithRemaining(stmt); @@ -1986,7 +2368,9 @@ private ParsingResult parseAndSplit(String schemaName, SqlFieldsQuery qry, int f .distributedJoinMode(distributedJoinMode(qry.isLocal(), qry.isDistributedJoins()))); try { - return new ParsingResult(prepared, newQry, remainingSql, split(prepared, newQry), + GridCacheTwoStepQuery twoStepQry = split(prepared, newQry); + + return new ParsingResult(prepared, newQry, remainingSql, twoStepQry, cachedQryKey, H2Utils.meta(stmt.getMetaData())); } catch (IgniteCheckedException e) { @@ -2046,6 +2430,63 @@ private GridCacheTwoStepQuery split(Prepared prepared, SqlFieldsQuery qry) throw return res; } + /** + * @param qry Sql fields query.autoStartTx(qry) + * @return {@code True} if need to start transaction. + */ + public boolean autoStartTx(SqlFieldsQuery qry) { + if (!mvccEnabled(ctx)) + return false; + + return qry instanceof SqlFieldsQueryEx && !((SqlFieldsQueryEx)qry).isAutoCommit() && tx(ctx) == null; + } + + /** {@inheritDoc} */ + @Override public UpdateSourceIterator prepareDistributedUpdate(GridCacheContext cctx, int[] ids, + int[] parts, String schema, String qry, Object[] params, int flags, + int pageSize, int timeout, AffinityTopologyVersion topVer, + MvccSnapshot mvccSnapshot, GridQueryCancel cancel) throws IgniteCheckedException { + + SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry); + + if (params != null) + fldsQry.setArgs(params); + + fldsQry.setEnforceJoinOrder(isFlagSet(flags, GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER)); + fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS); + fldsQry.setPageSize(pageSize); + fldsQry.setLocal(true); + + boolean loc = true; + + final boolean replicated = isFlagSet(flags, GridH2QueryRequest.FLAG_REPLICATED); + + GridCacheContext cctx0; + + if (!replicated + && !F.isEmpty(ids) + && (cctx0 = CU.firstPartitioned(cctx.shared(), ids)) != null + && cctx0.config().getQueryParallelism() > 1) { + fldsQry.setDistributedJoins(true); + + loc = false; + } + + Connection conn = connectionForSchema(schema); + + H2Utils.setupConnection(conn, false, fldsQry.isEnforceJoinOrder()); + + PreparedStatement stmt = preparedStatementWithParams(conn, fldsQry.getSql(), + F.asList(fldsQry.getArgs()), true); + + return dmlProc.prepareDistributedUpdate(schema, conn, stmt, fldsQry, backupFilter(topVer, parts), cancel, loc, + topVer, mvccSnapshot); + } + + private boolean isFlagSet(int flags, int flag) { + return (flags & flag) == flag; + } + /** * Run distributed query on detected set of partitions. * @param schemaName Schema name. @@ -2053,12 +2494,14 @@ private GridCacheTwoStepQuery split(Prepared prepared, SqlFieldsQuery qry) throw * @param twoStepQry Two-step query. * @param meta Metadata to set to cursor. * @param keepBinary Keep binary flag. + * @param startTx Start transaction flag. + * @param mvccTracker Query tracker. * @param cancel Cancel handler. * @return Cursor representing distributed query result. */ private FieldsQueryCursor> doRunDistributedQuery(String schemaName, SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, List meta, boolean keepBinary, - GridQueryCancel cancel) { + boolean startTx, MvccQueryTracker mvccTracker, GridQueryCancel cancel) { if (log.isDebugEnabled()) log.debug("Parsed query: `" + qry.getSql() + "` into two step query: " + twoStepQry); @@ -2080,8 +2523,8 @@ private FieldsQueryCursor> doRunDistributedQuery(String schemaName, SqlF } QueryCursorImpl> cursor = new QueryCursorImpl<>( - runQueryTwoStep(schemaName, twoStepQry, keepBinary, qry.isEnforceJoinOrder(), qry.getTimeout(), cancel, - qry.getArgs(), partitions, qry.isLazy()), cancel); + runQueryTwoStep(schemaName, twoStepQry, keepBinary, qry.isEnforceJoinOrder(), startTx, qry.getTimeout(), + cancel, qry.getArgs(), partitions, qry.isLazy(), mvccTracker), cancel); cursor.fieldsMeta(meta); @@ -2147,10 +2590,11 @@ public UpdateResult mapDistributedUpdate(String schemaName, SqlFieldsQuery fldsQ } /** - * @param cacheIds Caches identifiers. + * @param cacheIds Cache IDs. + * @param twoStepQry Query. * @throws IllegalStateException if segmented indices used with non-segmented indices. */ - private void checkCacheIndexSegmentation(Collection cacheIds) { + private void processCaches(List cacheIds, GridCacheTwoStepQuery twoStepQry) { if (cacheIds.isEmpty()) return; // Nothing to check @@ -2158,11 +2602,21 @@ private void checkCacheIndexSegmentation(Collection cacheIds) { int expectedParallelism = 0; - for (Integer cacheId : cacheIds) { + boolean mvccEnabled = false; + + for (int i = 0; i < cacheIds.size(); i++) { + Integer cacheId = cacheIds.get(i); + GridCacheContext cctx = sharedCtx.cacheContext(cacheId); assert cctx != null; + if (i == 0) + mvccEnabled = cctx.mvccEnabled(); + else if (cctx.mvccEnabled() != mvccEnabled) + throw new IllegalStateException("Using caches with different mvcc settings in same query is " + + "forbidden."); + if (!cctx.isPartitioned()) continue; @@ -2173,6 +2627,18 @@ else if (cctx.config().getQueryParallelism() != expectedParallelism) { "forbidden."); } } + + twoStepQry.mvccEnabled(mvccEnabled); + + if (twoStepQry.forUpdate()) { + if (cacheIds.size() != 1) + throw new IgniteSQLException("SELECT FOR UPDATE is supported only for queries " + + "that involve single transactional cache."); + + if (!mvccEnabled) + throw new IgniteSQLException("SELECT FOR UPDATE query requires transactional cache " + + "with MVCC enabled.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + } } /** @@ -2455,10 +2921,10 @@ private String dbTypeFromClass(Class cls, int precision, int scale) { /** * Gets collection of table for given schema name. * - * @param cacheName Schema name. + * @param cacheName Cache name. * @return Collection of table descriptors. */ - private Collection tables(String cacheName) { + Collection tables(String cacheName) { H2Schema s = schemas.get(schema(cacheName)); if (s == null) @@ -2487,28 +2953,32 @@ private Collection tables(String cacheName) { } /** - * Called periodically by {@link GridTimeoutProcessor} to clean up the {@link #stmtCache}. + * Called periodically by {@link GridTimeoutProcessor} to clean up the statement cache. */ private void cleanupStatementCache() { - long cur = U.currentTimeMillis(); + long now = U.currentTimeMillis(); - for (Iterator> it = stmtCache.entrySet().iterator(); it.hasNext(); ) { - Map.Entry entry = it.next(); + for (Iterator> it = conns.entrySet().iterator(); it.hasNext(); ) { + Map.Entry entry = it.next(); Thread t = entry.getKey(); - if (t.getState() == Thread.State.TERMINATED - || cur - entry.getValue().lastUsage() > STATEMENT_CACHE_THREAD_USAGE_TIMEOUT) + if (t.getState() == Thread.State.TERMINATED) { + U.close(entry.getValue(), log); + it.remove(); + } + else if (now - entry.getValue().statementCache().lastUsage() > STATEMENT_CACHE_THREAD_USAGE_TIMEOUT) + entry.getValue().clearStatementCache(); } } /** - * Called periodically by {@link GridTimeoutProcessor} to clean up the {@link #stmtCache}. + * Called periodically by {@link GridTimeoutProcessor} to clean up the {@link #conns}. */ private void cleanupConnections() { - for (Iterator> it = conns.entrySet().iterator(); it.hasNext(); ) { - Map.Entry entry = it.next(); + for (Iterator> it = conns.entrySet().iterator(); it.hasNext(); ) { + Map.Entry entry = it.next(); Thread t = entry.getKey(); @@ -2520,6 +2990,24 @@ private void cleanupConnections() { } } + /** + * Removes from cache and returns associated with current thread connection. + * @return Connection associated with current thread. + */ + public ThreadLocalObjectPool.Reusable detach() { + Thread key = Thread.currentThread(); + + ThreadLocalObjectPool.Reusable reusableConnection = connCache.get(); + + H2ConnectionWrapper connection = conns.remove(key); + + connCache.remove(); + + assert reusableConnection.object().connection() == connection.connection(); + + return reusableConnection; + } + /** * Rebuild indexes from hash index. * @@ -2535,7 +3023,7 @@ private void cleanupConnections() { SchemaIndexCacheVisitor visitor = new SchemaIndexCacheVisitorImpl(cctx); - visitor.visit(new RebuldIndexFromHashClosure(qryMgr)); + visitor.visit(new RebuildIndexFromHashClosure(qryMgr, cctx.mvccEnabled())); for (H2TableDescriptor tblDesc : tables(cacheName)) tblDesc.table().markRebuildFromHashInProgress(false); @@ -2885,7 +3373,7 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe mapQryExec.cancelLazyWorkers(); - for (Connection c : conns.values()) + for (H2ConnectionWrapper c : conns.values()) U.close(c, log); conns.clear(); @@ -2920,6 +3408,17 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe } } + /** {@inheritDoc} */ + @Override public void onClientDisconnect() throws IgniteCheckedException { + if (!mvccEnabled(ctx)) + return; + + GridNearTxLocal tx = tx(ctx); + + if (tx != null) + doRollback(tx); + } + /** {@inheritDoc} */ @Override public void registerCache(String cacheName, String schemaName, GridCacheContext cctx) throws IgniteCheckedException { @@ -2984,7 +3483,7 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe } } - stmtCache.clear(); + conns.values().forEach(H2ConnectionWrapper::clearStatementCache); for (H2TableDescriptor tbl : rmvTbls) { for (Index idx : tbl.table().getIndexes()) @@ -3144,14 +3643,14 @@ private int bindPartitionInfoParameter(CacheQueryPartitionInfo partInfo, Object[ @Override public void cancelAllQueries() { mapQryExec.cancelLazyWorkers(); - for (Connection c : conns.values()) + for (H2ConnectionWrapper c : conns.values()) U.close(c, log); } /** * @return Per-thread connections. */ - public Map perThreadConnections() { + public Map perThreadConnections() { return conns; } @@ -3188,7 +3687,7 @@ public Map perThreadConnections() { //Prohibit usage indices with different numbers of segments in same query. List cacheIds = new ArrayList<>(caches0); - checkCacheIndexSegmentation(cacheIds); + processCaches(cacheIds, twoStepQry); return cacheIds; } @@ -3214,22 +3713,4 @@ private boolean hasSystemViews(GridCacheTwoStepQuery twoStepQry) { private interface ClIter extends AutoCloseable, Iterator { // No-op. } - - /** */ - private static class RebuldIndexFromHashClosure implements SchemaIndexCacheVisitorClosure { - /** */ - private final GridCacheQueryManager qryMgr; - - /** - * @param qryMgr Query manager. - */ - RebuldIndexFromHashClosure(GridCacheQueryManager qryMgr) { - this.qryMgr = qryMgr; - } - - /** {@inheritDoc} */ - @Override public void apply(CacheDataRow row) throws IgniteCheckedException { - qryMgr.store(row, null, false); - } - } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/NearResultSetEnlistFuture.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/NearResultSetEnlistFuture.java new file mode 100644 index 0000000000000..18564302d806e --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/NearResultSetEnlistFuture.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2; + +import java.sql.ResultSet; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class NearResultSetEnlistFuture extends GridNearTxQueryResultsEnlistFuture implements ResultSetEnlistFuture { + /** */ + private static final long serialVersionUID = 877907044489718378L; + + /** + * @param nearNodeId Near node ID. + * @param nearLockVer Near lock version. + * @param mvccSnapshot Mvcc snapshot. + * @param threadId Thread ID. + * @param nearFutId Near future id. + * @param nearMiniId Near mini future id. + * @param parts Partitions. + * @param tx Transaction. + * @param timeout Lock acquisition timeout. + * @param cctx Cache context. + * @param rs Result set to process. + */ + public NearResultSetEnlistFuture(UUID nearNodeId, GridCacheVersion nearLockVer, MvccSnapshot mvccSnapshot, + long threadId, IgniteUuid nearFutId, int nearMiniId, @Nullable int[] parts, GridDhtTxLocalAdapter tx, + long timeout, GridCacheContext cctx, ResultSet rs) { + super(cctx, (GridNearTxLocal)tx, timeout, ResultSetEnlistFuture.createIterator(rs), 0, true); + } + + + /** {@inheritDoc} */ + @Override public void onError(Throwable error) { + onDone(error); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementEx.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementEx.java new file mode 100644 index 0000000000000..d254971d5187c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementEx.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2; + +import java.sql.PreparedStatement; +import java.util.concurrent.atomic.AtomicInteger; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public interface PreparedStatementEx extends PreparedStatement { + /** */ + static final AtomicInteger metaIdGenerator = new AtomicInteger(); + + /** */ + static final int INVOLVED_CACHES = metaIdGenerator.getAndIncrement(); + + /** + * @param id Metadata key. + * @return Attached metadata. + */ + @Nullable T meta(int id); + + /** + * @param id Metadata key. + * @param metaObj Metadata object. + */ + void putMeta(int id, Object metaObj); +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExImpl.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExImpl.java new file mode 100644 index 0000000000000..922c1ab29d42c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExImpl.java @@ -0,0 +1,648 @@ +/* + * 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.InputStream; +import java.io.Reader; +import java.math.BigDecimal; +import java.net.URL; +import java.sql.Array; +import java.sql.Blob; +import java.sql.Clob; +import java.sql.Connection; +import java.sql.Date; +import java.sql.NClob; +import java.sql.ParameterMetaData; +import java.sql.PreparedStatement; +import java.sql.Ref; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.RowId; +import java.sql.SQLException; +import java.sql.SQLType; +import java.sql.SQLWarning; +import java.sql.SQLXML; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.Calendar; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * PreparedStatement with extended capability to store additional meta information. + */ +@SuppressWarnings("unchecked") +final class PreparedStatementExImpl implements PreparedStatementEx { + /** */ + private final PreparedStatement delegate; + + /** */ + private Object[] meta = null; + + /** + * @param delegate Wrapped statement. + */ + public PreparedStatementExImpl(PreparedStatement delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public ResultSet executeQuery() throws SQLException { + return delegate.executeQuery(); + } + + /** {@inheritDoc} */ + @Override public int executeUpdate() throws SQLException { + return delegate.executeUpdate(); + } + + /** {@inheritDoc} */ + @Override public void setNull(int parameterIndex, int sqlType) throws SQLException { + delegate.setNull(parameterIndex, sqlType); + } + + /** {@inheritDoc} */ + @Override public void setBoolean(int parameterIndex, boolean x) throws SQLException { + delegate.setBoolean(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setByte(int parameterIndex, byte x) throws SQLException { + delegate.setByte(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setShort(int parameterIndex, short x) throws SQLException { + delegate.setShort(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setInt(int parameterIndex, int x) throws SQLException { + delegate.setInt(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setLong(int parameterIndex, long x) throws SQLException { + delegate.setLong(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setFloat(int parameterIndex, float x) throws SQLException { + delegate.setFloat(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setDouble(int parameterIndex, double x) throws SQLException { + delegate.setDouble(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { + delegate.setBigDecimal(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setString(int parameterIndex, String x) throws SQLException { + delegate.setString(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setBytes(int parameterIndex, byte[] x) throws SQLException { + delegate.setBytes(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setDate(int parameterIndex, Date x) throws SQLException { + delegate.setDate(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setTime(int parameterIndex, Time x) throws SQLException { + delegate.setTime(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { + delegate.setTimestamp(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { + delegate.setAsciiStream(parameterIndex, x, length); + } + + /** {@inheritDoc} */ + @Deprecated + @Override public void setUnicodeStream(int parameterIndex, InputStream x, int length) throws SQLException { + delegate.setUnicodeStream(parameterIndex, x, length); + } + + /** {@inheritDoc} */ + @Override public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { + delegate.setBinaryStream(parameterIndex, x, length); + } + + /** {@inheritDoc} */ + @Override public void clearParameters() throws SQLException { + delegate.clearParameters(); + } + + /** {@inheritDoc} */ + @Override public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { + delegate.setObject(parameterIndex, x, targetSqlType); + } + + /** {@inheritDoc} */ + @Override public void setObject(int parameterIndex, Object x) throws SQLException { + delegate.setObject(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public boolean execute() throws SQLException { + return delegate.execute(); + } + + /** {@inheritDoc} */ + @Override public void addBatch() throws SQLException { + delegate.addBatch(); + } + + /** {@inheritDoc} */ + @Override public void setCharacterStream(int parameterIndex, Reader reader, int length) throws SQLException { + delegate.setCharacterStream(parameterIndex, reader, length); + } + + /** {@inheritDoc} */ + @Override public void setRef(int parameterIndex, Ref x) throws SQLException { + delegate.setRef(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setBlob(int parameterIndex, Blob x) throws SQLException { + delegate.setBlob(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setClob(int parameterIndex, Clob x) throws SQLException { + delegate.setClob(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setArray(int parameterIndex, Array x) throws SQLException { + delegate.setArray(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public ResultSetMetaData getMetaData() throws SQLException { + return delegate.getMetaData(); + } + + /** {@inheritDoc} */ + @Override public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { + delegate.setDate(parameterIndex, x, cal); + } + + /** {@inheritDoc} */ + @Override public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { + delegate.setTime(parameterIndex, x, cal); + } + + /** {@inheritDoc} */ + @Override public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { + delegate.setTimestamp(parameterIndex, x, cal); + } + + /** {@inheritDoc} */ + @Override public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException { + delegate.setNull(parameterIndex, sqlType, typeName); + } + + /** {@inheritDoc} */ + @Override public void setURL(int parameterIndex, URL x) throws SQLException { + delegate.setURL(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public ParameterMetaData getParameterMetaData() throws SQLException { + return delegate.getParameterMetaData(); + } + + /** {@inheritDoc} */ + @Override public void setRowId(int parameterIndex, RowId x) throws SQLException { + delegate.setRowId(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setNString(int parameterIndex, String value) throws SQLException { + delegate.setNString(parameterIndex, value); + } + + /** {@inheritDoc} */ + @Override public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException { + delegate.setNCharacterStream(parameterIndex, value, length); + } + + /** {@inheritDoc} */ + @Override public void setNClob(int parameterIndex, NClob value) throws SQLException { + delegate.setNClob(parameterIndex, value); + } + + /** {@inheritDoc} */ + @Override public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { + delegate.setClob(parameterIndex, reader, length); + } + + /** {@inheritDoc} */ + @Override public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException { + delegate.setBlob(parameterIndex, inputStream, length); + } + + /** {@inheritDoc} */ + @Override public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { + delegate.setNClob(parameterIndex, reader, length); + } + + /** {@inheritDoc} */ + @Override public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { + delegate.setSQLXML(parameterIndex, xmlObject); + } + + @Override + public void setObject(int parameterIndex, Object x, int targetSqlType, int scaleOrLength) throws SQLException { + delegate.setObject(parameterIndex, x, targetSqlType, scaleOrLength); + } + + /** {@inheritDoc} */ + @Override public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { + delegate.setAsciiStream(parameterIndex, x, length); + } + + /** {@inheritDoc} */ + @Override public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { + delegate.setBinaryStream(parameterIndex, x, length); + } + + /** {@inheritDoc} */ + @Override public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException { + delegate.setCharacterStream(parameterIndex, reader, length); + } + + /** {@inheritDoc} */ + @Override public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { + delegate.setAsciiStream(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { + delegate.setBinaryStream(parameterIndex, x); + } + + /** {@inheritDoc} */ + @Override public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { + delegate.setCharacterStream(parameterIndex, reader); + } + + /** {@inheritDoc} */ + @Override public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { + delegate.setNCharacterStream(parameterIndex, value); + } + + /** {@inheritDoc} */ + @Override public void setClob(int parameterIndex, Reader reader) throws SQLException { + delegate.setClob(parameterIndex, reader); + } + + /** {@inheritDoc} */ + @Override public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { + delegate.setBlob(parameterIndex, inputStream); + } + + /** {@inheritDoc} */ + @Override public void setNClob(int parameterIndex, Reader reader) throws SQLException { + delegate.setNClob(parameterIndex, reader); + } + + /** {@inheritDoc} */ + @Override public void setObject(int parameterIndex, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException { + delegate.setObject(parameterIndex, x, targetSqlType, scaleOrLength); + } + + /** {@inheritDoc} */ + @Override public void setObject(int parameterIndex, Object x, SQLType targetSqlType) throws SQLException { + delegate.setObject(parameterIndex, x, targetSqlType); + } + + /** {@inheritDoc} */ + @Override public long executeLargeUpdate() throws SQLException { + return delegate.executeLargeUpdate(); + } + + /** {@inheritDoc} */ + @Override public ResultSet executeQuery(String sql) throws SQLException { + return delegate.executeQuery(sql); + } + + /** {@inheritDoc} */ + @Override public int executeUpdate(String sql) throws SQLException { + return delegate.executeUpdate(sql); + } + + /** {@inheritDoc} */ + @Override public void close() throws SQLException { + delegate.close(); + } + + /** {@inheritDoc} */ + @Override public int getMaxFieldSize() throws SQLException { + return delegate.getMaxFieldSize(); + } + + /** {@inheritDoc} */ + @Override public void setMaxFieldSize(int max) throws SQLException { + delegate.setMaxFieldSize(max); + } + + /** {@inheritDoc} */ + @Override public int getMaxRows() throws SQLException { + return delegate.getMaxRows(); + } + + /** {@inheritDoc} */ + @Override public void setMaxRows(int max) throws SQLException { + delegate.setMaxRows(max); + } + + /** {@inheritDoc} */ + @Override public void setEscapeProcessing(boolean enable) throws SQLException { + delegate.setEscapeProcessing(enable); + } + + /** {@inheritDoc} */ + @Override public int getQueryTimeout() throws SQLException { + return delegate.getQueryTimeout(); + } + + /** {@inheritDoc} */ + @Override public void setQueryTimeout(int seconds) throws SQLException { + delegate.setQueryTimeout(seconds); + } + + /** {@inheritDoc} */ + @Override public void cancel() throws SQLException { + delegate.cancel(); + } + + /** {@inheritDoc} */ + @Override public SQLWarning getWarnings() throws SQLException { + return delegate.getWarnings(); + } + + /** {@inheritDoc} */ + @Override public void clearWarnings() throws SQLException { + delegate.clearWarnings(); + } + + /** {@inheritDoc} */ + @Override public void setCursorName(String name) throws SQLException { + delegate.setCursorName(name); + } + + /** {@inheritDoc} */ + @Override public boolean execute(String sql) throws SQLException { + return delegate.execute(sql); + } + + /** {@inheritDoc} */ + @Override public ResultSet getResultSet() throws SQLException { + return delegate.getResultSet(); + } + + /** {@inheritDoc} */ + @Override public int getUpdateCount() throws SQLException { + return delegate.getUpdateCount(); + } + + /** {@inheritDoc} */ + @Override public boolean getMoreResults() throws SQLException { + return delegate.getMoreResults(); + } + + /** {@inheritDoc} */ + @Override public int getFetchDirection() throws SQLException { + return delegate.getFetchDirection(); + } + + /** {@inheritDoc} */ + @Override public void setFetchDirection(int direction) throws SQLException { + delegate.setFetchDirection(direction); + } + + /** {@inheritDoc} */ + @Override public int getFetchSize() throws SQLException { + return delegate.getFetchSize(); + } + + /** {@inheritDoc} */ + @Override public void setFetchSize(int rows) throws SQLException { + delegate.setFetchSize(rows); + } + + /** {@inheritDoc} */ + @Override public int getResultSetConcurrency() throws SQLException { + return delegate.getResultSetConcurrency(); + } + + /** {@inheritDoc} */ + @Override public int getResultSetType() throws SQLException { + return delegate.getResultSetType(); + } + + /** {@inheritDoc} */ + @Override public void addBatch(String sql) throws SQLException { + delegate.addBatch(sql); + } + + /** {@inheritDoc} */ + @Override public void clearBatch() throws SQLException { + delegate.clearBatch(); + } + + /** {@inheritDoc} */ + @Override public int[] executeBatch() throws SQLException { + return delegate.executeBatch(); + } + + /** {@inheritDoc} */ + @Override public Connection getConnection() throws SQLException { + return delegate.getConnection(); + } + + /** {@inheritDoc} */ + @Override public boolean getMoreResults(int current) throws SQLException { + return delegate.getMoreResults(current); + } + + /** {@inheritDoc} */ + @Override public ResultSet getGeneratedKeys() throws SQLException { + return delegate.getGeneratedKeys(); + } + + /** {@inheritDoc} */ + @Override public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException { + return delegate.executeUpdate(sql, autoGeneratedKeys); + } + + /** {@inheritDoc} */ + @Override public int executeUpdate(String sql, int[] columnIndexes) throws SQLException { + return delegate.executeUpdate(sql, columnIndexes); + } + + /** {@inheritDoc} */ + @Override public int executeUpdate(String sql, String[] columnNames) throws SQLException { + return delegate.executeUpdate(sql, columnNames); + } + + /** {@inheritDoc} */ + @Override public boolean execute(String sql, int autoGeneratedKeys) throws SQLException { + return delegate.execute(sql, autoGeneratedKeys); + } + + /** {@inheritDoc} */ + @Override public boolean execute(String sql, int[] columnIndexes) throws SQLException { + return delegate.execute(sql, columnIndexes); + } + + /** {@inheritDoc} */ + @Override public boolean execute(String sql, String[] columnNames) throws SQLException { + return delegate.execute(sql, columnNames); + } + + /** {@inheritDoc} */ + @Override public int getResultSetHoldability() throws SQLException { + return delegate.getResultSetHoldability(); + } + + /** {@inheritDoc} */ + @Override public boolean isClosed() throws SQLException { + return delegate.isClosed(); + } + + /** {@inheritDoc} */ + @Override public boolean isPoolable() throws SQLException { + return delegate.isPoolable(); + } + + /** {@inheritDoc} */ + @Override public void setPoolable(boolean poolable) throws SQLException { + delegate.setPoolable(poolable); + } + + /** {@inheritDoc} */ + @Override public void closeOnCompletion() throws SQLException { + delegate.closeOnCompletion(); + } + + /** {@inheritDoc} */ + @Override public boolean isCloseOnCompletion() throws SQLException { + return delegate.isCloseOnCompletion(); + } + + /** {@inheritDoc} */ + @Override public long getLargeUpdateCount() throws SQLException { + return delegate.getLargeUpdateCount(); + } + + /** {@inheritDoc} */ + @Override public long getLargeMaxRows() throws SQLException { + return delegate.getLargeMaxRows(); + } + + /** {@inheritDoc} */ + @Override public void setLargeMaxRows(long max) throws SQLException { + delegate.setLargeMaxRows(max); + } + + /** {@inheritDoc} */ + @Override public long[] executeLargeBatch() throws SQLException { + return delegate.executeLargeBatch(); + } + + /** {@inheritDoc} */ + @Override public long executeLargeUpdate(String sql) throws SQLException { + return delegate.executeLargeUpdate(sql); + } + + /** {@inheritDoc} */ + @Override public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException { + return delegate.executeLargeUpdate(sql, autoGeneratedKeys); + } + + /** {@inheritDoc} */ + @Override public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException { + return delegate.executeLargeUpdate(sql, columnIndexes); + } + + /** {@inheritDoc} */ + @Override public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException { + return delegate.executeLargeUpdate(sql, columnNames); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public T unwrap(Class iface) throws SQLException { + if (iface == PreparedStatementExImpl.class || iface == PreparedStatementEx.class) + return (T)this; + + return delegate.unwrap(iface); + } + + /** {@inheritDoc} */ + @Override public boolean isWrapperFor(Class iface) throws SQLException { + return iface == PreparedStatementExImpl.class + || iface == PreparedStatementEx.class + || delegate.isWrapperFor(iface); + } + + /** {@inheritDoc} */ + @Override public @Nullable T meta(int id) { + return meta != null && id < meta.length ? (T)meta[id] : null; + } + + /** {@inheritDoc} */ + @Override public void putMeta(int id, Object metaObj) { + if (meta == null) + meta = new Object[id + 1]; + else if (meta.length <= id) + meta = Arrays.copyOf(meta, id + 1); + + meta[id] = metaObj; + } + + /** + * + * @param stmt Prepared statement to wrap. + * @return Wrapped statement. + */ + public static PreparedStatement wrap(@NotNull PreparedStatement stmt) { + if (stmt.getClass() == PreparedStatementExImpl.class) + return stmt; + + return new PreparedStatementExImpl(stmt); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/RebuildIndexFromHashClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/RebuildIndexFromHashClosure.java new file mode 100644 index 0000000000000..b635eaca8892b --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/RebuildIndexFromHashClosure.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.query.h2; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; +import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure; + +/** */ +class RebuildIndexFromHashClosure implements SchemaIndexCacheVisitorClosure { + /** */ + private final GridCacheQueryManager qryMgr; + + /** MVCC status flag. */ + private final boolean mvccEnabled; + + /** + * @param qryMgr Query manager. + * @param mvccEnabled MVCC status flag. + */ + RebuildIndexFromHashClosure(GridCacheQueryManager qryMgr, boolean mvccEnabled) { + this.qryMgr = qryMgr; + this.mvccEnabled = mvccEnabled; + } + + /** {@inheritDoc} */ + @Override public void apply(CacheDataRow row) throws IgniteCheckedException { + // prevRowAvailable is always true with MVCC on, and always false *on index rebuild* with MVCC off. + qryMgr.store(row, null, mvccEnabled); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ResultSetEnlistFuture.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ResultSetEnlistFuture.java new file mode 100644 index 0000000000000..ee1c0fa43d974 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ResultSetEnlistFuture.java @@ -0,0 +1,136 @@ +/* + * 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.sql.ResultSet; +import java.sql.SQLException; +import java.util.NoSuchElementException; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.DhtLockFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.EnlistOperation; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.processors.query.UpdateSourceIterator; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +/** + * Future to process whole local result set of SELECT FOR UPDATE query. + */ +public interface ResultSetEnlistFuture extends DhtLockFuture { + /** + * @param rs Result set. + * @return Update source. + */ + static UpdateSourceIterator createIterator(ResultSet rs) { + return new ResultSetUpdateSourceIteratorWrapper(rs); + } + + /** */ + void init(); + + /** + * + * @param nearNodeId Near node ID. + * @param nearLockVer Near lock version. + * @param mvccSnapshot Mvcc snapshot. + * @param threadId Thread ID. + * @param nearFutId Near future id. + * @param nearMiniId Near mini future id. + * @param parts Partitions. + * @param tx Transaction. + * @param timeout Lock acquisition timeout. + * @param cctx Cache context. + * @param rs Result set to process. + * @return Result set enlist future. + */ + static ResultSetEnlistFuture future(UUID nearNodeId, GridCacheVersion nearLockVer, + MvccSnapshot mvccSnapshot, long threadId, IgniteUuid nearFutId, int nearMiniId, @Nullable int[] parts, + GridDhtTxLocalAdapter tx, long timeout, GridCacheContext cctx, ResultSet rs) { + + if (tx.near()) + return new NearResultSetEnlistFuture(nearNodeId, nearLockVer, mvccSnapshot, threadId, nearFutId, nearMiniId, parts, tx, timeout, cctx, rs); + else + return new DhtResultSetEnlistFuture(nearNodeId, nearLockVer, mvccSnapshot, threadId, nearFutId, nearMiniId, parts, tx, timeout, cctx, rs); + } + + /** + * + */ + public static class ResultSetUpdateSourceIteratorWrapper implements UpdateSourceIterator { + /** */ + private static final long serialVersionUID = -8745196216234843471L; + + /** */ + private final ResultSet rs; + + /** */ + private Boolean hasNext; + + /** */ + private int keyColIdx; + + /** + * @param rs Result set. + */ + public ResultSetUpdateSourceIteratorWrapper(ResultSet rs) { + this.rs = rs; + keyColIdx = -1; + } + + /** {@inheritDoc} */ + @Override public EnlistOperation operation() { + return EnlistOperation.LOCK; + } + + /** {@inheritDoc} */ + @Override public boolean hasNextX() { + try { + if (hasNext == null) + hasNext = rs.next(); + + return hasNext; + } + catch (SQLException e) { + throw new IgniteSQLException(e); + } + } + + /** {@inheritDoc} */ + @Override public Object nextX() { + if (!hasNextX()) + throw new NoSuchElementException(); + + try { + if (keyColIdx == -1) + keyColIdx = rs.getMetaData().getColumnCount(); + + return rs.getObject(keyColIdx); + } + catch (SQLException e) { + throw new IgniteSQLException(e); + } + finally { + hasNext = null; + } + } + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.java new file mode 100644 index 0000000000000..25daa23e38b8a --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPool.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.query.h2; + +import java.util.LinkedList; +import java.util.Queue; +import java.util.function.Supplier; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Special pool for managing limited number objects for further reuse. + * This pool maintains separate object bag for each thread by means of {@link ThreadLocal}. + *

    + * If object is borrowed on one thread and recycled on different then it will be returned to + * recycling thread bag. For thread-safe use either pooled objects should be thread-safe or + * happens-before should be established between borrowing object and subsequent recycling. + * + * @param pooled objects type + */ +public final class ThreadLocalObjectPool { + /** + * Wrapper for a pooled object with capability to return the object to a pool. + * + * @param enclosed object type + */ + public static class Reusable { + /** */ + private final ThreadLocalObjectPool pool; + /** */ + private final T object; + + /** */ + private Reusable(ThreadLocalObjectPool pool, T object) { + this.pool = pool; + this.object = object; + } + + /** + * @return enclosed object + */ + public T object() { + return object; + } + + /** + * Returns an object to a pool or closes it if the pool is already full. + */ + public void recycle() { + Queue> bag = pool.bag.get(); + if (bag.size() < pool.poolSize) + bag.add(this); + else + U.closeQuiet(object); + } + } + + /** */ + private final Supplier objectFactory; + /** */ + private final ThreadLocal>> bag = ThreadLocal.withInitial(LinkedList::new); + /** */ + private final int poolSize; + + /** + * @param objectFactory factory used for new objects creation + * @param poolSize number of objects which pool can contain + */ + public ThreadLocalObjectPool(Supplier objectFactory, int poolSize) { + this.objectFactory = objectFactory; + this.poolSize = poolSize; + } + + /** + * Picks an object from the pool if one is present or creates new one otherwise. + * Returns an object wrapper which could be returned to the pool. + * + * @return reusable object wrapper + */ + public Reusable borrow() { + Reusable pooled = bag.get().poll(); + return pooled != null ? pooled : new Reusable<>(this, objectFactory.get()); + } + + /** Visible for test */ + int bagSize() { + return bag.get().size(); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java index 546f5bb2e9a47..5d877cdbc2fce 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java @@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.database; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -26,9 +27,12 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; @@ -84,31 +88,30 @@ public H2PkHashIndex( /** {@inheritDoc} */ @Override public Cursor find(Session ses, final SearchRow lower, final SearchRow upper) { - IndexingQueryFilter f = threadLocalFilter(); - IndexingQueryCacheFilter p = null; + IndexingQueryCacheFilter filter = null; + MvccSnapshot mvccSnapshot = null; - if (f != null) { - String cacheName = getTable().cacheName(); + GridH2QueryContext qctx = GridH2QueryContext.get(); - p = f.forCache(cacheName); + if (qctx != null) { + IndexingQueryFilter f = qctx.filter(); + filter = f != null ? f.forCache(getTable().cacheName()) : null; + mvccSnapshot = qctx.mvccSnapshot(); } - KeyCacheObject lowerObj = null; - KeyCacheObject upperObj = null; + assert !cctx.mvccEnabled() || mvccSnapshot != null; - if (lower != null) - lowerObj = cctx.toCacheKeyObject(lower.getValue(0).getObject()); - - if (upper != null) - upperObj = cctx.toCacheKeyObject(upper.getValue(0).getObject()); + KeyCacheObject lowerObj = lower != null ? cctx.toCacheKeyObject(lower.getValue(0).getObject()) : null; + KeyCacheObject upperObj = upper != null ? cctx.toCacheKeyObject(upper.getValue(0).getObject()) : null; try { - List> cursors = new ArrayList<>(); + Collection> cursors = new ArrayList<>(); for (IgniteCacheOffheapManager.CacheDataStore store : cctx.offheap().cacheDataStores()) - cursors.add(store.cursor(cctx.cacheId(), lowerObj, upperObj)); + if (filter == null || filter.applyPartition(store.partId())) + cursors.add(store.cursor(cctx.cacheId(), lowerObj, upperObj, null, mvccSnapshot)); - return new H2Cursor(new CompositeGridCursor<>(cursors.iterator()), p); + return new H2Cursor(cursors.iterator()); } catch (IgniteCheckedException e) { throw DbException.convert(e); @@ -124,7 +127,6 @@ public H2PkHashIndex( @SuppressWarnings("StatementWithEmptyBody") @Override public GridH2Row put(GridH2Row row) { // Should not be called directly. Rows are inserted into underlying cache data stores. - assert false; throw DbException.getUnsupportedException("put"); @@ -192,28 +194,29 @@ public H2PkHashIndex( */ private class H2Cursor implements Cursor { /** */ - final GridCursor cursor; + private final GridH2RowDescriptor desc; /** */ - final IndexingQueryCacheFilter filter; + private final Iterator> iter; + + /** */ + private GridCursor curr; /** - * @param cursor Cursor. - * @param filter Filter. + * @param iter Cursors iterator. */ - private H2Cursor(GridCursor cursor, IndexingQueryCacheFilter filter) { - assert cursor != null; + private H2Cursor(Iterator> iter) { + assert iter != null; - this.cursor = cursor; - this.filter = filter; + this.iter = iter; + + desc = tbl.rowDescriptor(); } /** {@inheritDoc} */ @Override public Row get() { try { - CacheDataRow dataRow = cursor.get(); - - return tbl.rowDescriptor().createRow(dataRow); + return desc.createRow(curr.get()); } catch (IgniteCheckedException e) { throw DbException.convert(e); @@ -228,13 +231,13 @@ private H2Cursor(GridCursor cursor, IndexingQueryCacheFi /** {@inheritDoc} */ @Override public boolean next() { try { - while (cursor.next()) { - if (filter == null) - return true; + if (curr != null && curr.next()) + return true; - CacheDataRow dataRow = cursor.get(); + while (iter.hasNext()) { + curr = iter.next(); - if (filter.applyPartition(dataRow.partition())) + if (curr.next()) return true; } @@ -250,45 +253,4 @@ private H2Cursor(GridCursor cursor, IndexingQueryCacheFi throw DbException.getUnsupportedException("previous"); } } - - /** - * - */ - private static class CompositeGridCursor implements GridCursor { - /** */ - private final Iterator> iter; - - /** */ - private GridCursor curr; - - /** - * - */ - public CompositeGridCursor(Iterator> iter) { - this.iter = iter; - - if (iter.hasNext()) - curr = iter.next(); - } - - /** {@inheritDoc} */ - @Override public boolean next() throws IgniteCheckedException { - if (curr.next()) - return true; - - while (iter.hasNext()) { - curr = iter.next(); - - if (curr.next()) - return true; - } - - return false; - } - - /** {@inheritDoc} */ - @Override public T get() throws IgniteCheckedException { - return curr.get(); - } - } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java index 40b9b0a6e0c55..724de7ee8a40f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java @@ -18,9 +18,10 @@ package org.apache.ignite.internal.processors.query.h2.database; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor; @@ -60,17 +61,23 @@ public GridH2Row getRow(long link) throws IgniteCheckedException { rowBuilder.initFromLink(cctx.group(), CacheDataRowAdapter.RowData.FULL); - GridH2Row row; - - try { - row = rowDesc.createRow(rowBuilder); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } + GridH2Row row = rowDesc.createRow(rowBuilder); assert row.version() != null; return row; } + + /** + * @param link Link. + * @param mvccCrdVer Mvcc coordinator version. + * @param mvccCntr Mvcc counter. + * @param mvccOpCntr Mvcc operation counter. + * @return Row. + * @throws IgniteCheckedException If failed. + */ + public GridH2Row getMvccRow(long link, long mvccCrdVer, long mvccCntr, int mvccOpCntr) throws IgniteCheckedException { + return rowDesc.createRow(new MvccDataRow(cctx.group(),0, link, + PageIdUtils.partId(PageIdUtils.pageId(link)),null, mvccCrdVer, mvccCntr, mvccOpCntr)); + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java index 424969e6484d2..ce40df0039c7f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java @@ -21,9 +21,9 @@ import java.util.List; import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO; @@ -35,8 +35,9 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO; import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; import org.h2.result.SearchRow; import org.h2.table.IndexColumn; import org.h2.value.Value; @@ -44,7 +45,7 @@ /** */ -public abstract class H2Tree extends BPlusTree { +public abstract class H2Tree extends BPlusTree { /** */ private final H2RowFactory rowStore; @@ -60,6 +61,9 @@ public abstract class H2Tree extends BPlusTree { /** */ private final int[] columnIds; + /** */ + private final boolean mvccEnabled; + /** */ private final Comparator comp = new Comparator() { @Override public int compare(Value o1, Value o2) { @@ -82,6 +86,7 @@ public abstract class H2Tree extends BPlusTree { * @param metaPageId Meta page ID. * @param initNew Initialize new index. * @param rowCache Row cache. + * @param mvccEnabled Mvcc flag. * @param failureProcessor if the tree is corrupted. * @throws IgniteCheckedException If failed. */ @@ -98,6 +103,7 @@ protected H2Tree( IndexColumn[] cols, List inlineIdxs, int inlineSize, + boolean mvccEnabled, @Nullable H2RowCache rowCache, @Nullable FailureProcessor failureProcessor ) throws IgniteCheckedException { @@ -109,6 +115,7 @@ protected H2Tree( } this.inlineSize = inlineSize; + this.mvccEnabled = mvccEnabled; assert rowStore != null; @@ -121,9 +128,9 @@ protected H2Tree( for (int i = 0; i < cols.length; i++) columnIds[i] = cols[i].column.getColumnId(); - this.rowCache = rowCache; + setIos(H2ExtrasInnerIO.getVersions(inlineSize, mvccEnabled), H2ExtrasLeafIO.getVersions(inlineSize, mvccEnabled)); - setIos(H2ExtrasInnerIO.getVersions(inlineSize), H2ExtrasLeafIO.getVersions(inlineSize)); + this.rowCache = rowCache; initTree(initNew, inlineSize); } @@ -152,21 +159,34 @@ public GridH2Row createRowFromLink(long link) throws IgniteCheckedException { return rowStore.getRow(link); } - /** {@inheritDoc} */ - @Override protected GridH2Row getRow(BPlusIO io, long pageAddr, int idx, Object filter) - throws IgniteCheckedException { - if (filter != null) { - // Filter out not interesting partitions without deserializing the row. - IndexingQueryCacheFilter filter0 = (IndexingQueryCacheFilter)filter; + /** + * Create row from link. + * + * @param link Link. + * @param mvccOpCntr + * @return Row. + * @throws IgniteCheckedException if failed. + */ + public GridH2Row createRowFromLink(long link, long mvccCrdVer, long mvccCntr, int mvccOpCntr) throws IgniteCheckedException { + if (rowCache != null) { + GridH2Row row = rowCache.get(link); - long link = ((H2RowLinkIO)io).getLink(pageAddr, idx); + if (row == null) { + row = rowStore.getMvccRow(link, mvccCrdVer, mvccCntr, mvccOpCntr); - int part = PageIdUtils.partId(PageIdUtils.pageId(link)); + if (row instanceof GridH2KeyValueRowOnheap) + rowCache.put((GridH2KeyValueRowOnheap)row); + } - if (!filter0.applyPartition(part)) - return null; + return row; } + else + return rowStore.getMvccRow(link, mvccCrdVer, mvccCntr, mvccOpCntr); + } + /** {@inheritDoc} */ + @Override public GridH2Row getRow(BPlusIO io, long pageAddr, int idx, Object ignore) + throws IgniteCheckedException { return (GridH2Row)io.getLookupRow(this, pageAddr, idx); } @@ -206,8 +226,8 @@ private int getMetaInlineSize() throws IgniteCheckedException { /** {@inheritDoc} */ @SuppressWarnings("ForLoopReplaceableByForEach") - @Override protected int compare(BPlusIO io, long pageAddr, int idx, - SearchRow row) throws IgniteCheckedException { + @Override protected int compare(BPlusIO io, long pageAddr, int idx, + GridH2SearchRow row) throws IgniteCheckedException { if (inlineSize() == 0) return compareRows(getRow(io, pageAddr, idx), row); else { @@ -242,7 +262,7 @@ private int getMetaInlineSize() throws IgniteCheckedException { } if (lastIdxUsed == cols.length) - return 0; + return mvccCompare((H2RowLinkIO)io, pageAddr, idx, row); SearchRow rowData = getRow(io, pageAddr, idx); @@ -254,7 +274,7 @@ private int getMetaInlineSize() throws IgniteCheckedException { if (v2 == null) { // Can't compare further. - return 0; + return mvccCompare((H2RowLinkIO)io, pageAddr, idx, row); } Value v1 = rowData.getValue(idx0); @@ -265,7 +285,7 @@ private int getMetaInlineSize() throws IgniteCheckedException { return InlineIndexHelper.fixSort(c, col.sortType); } - return 0; + return mvccCompare((H2RowLinkIO)io, pageAddr, idx, row); } } @@ -276,7 +296,8 @@ private int getMetaInlineSize() throws IgniteCheckedException { * @param r2 Row 2. * @return Compare result: see {@link Comparator#compare(Object, Object)} for values. */ - public int compareRows(SearchRow r1, SearchRow r2) { + public int compareRows(GridH2SearchRow r1, GridH2SearchRow r2) { + assert !mvccEnabled || r2.indexSearchRow() || MvccUtils.mvccVersionIsValid(r2.mvccCoordinatorVersion(), r2.mvccCounter()) : r2; if (r1 == r2) return 0; @@ -288,7 +309,7 @@ public int compareRows(SearchRow r1, SearchRow r2) { if (v1 == null || v2 == null) { // Can't compare further. - return 0; + return mvccCompare(r1, r2); } int c = compareValues(v1, v2); @@ -297,7 +318,47 @@ public int compareRows(SearchRow r1, SearchRow r2) { return InlineIndexHelper.fixSort(c, cols[i].sortType); } - return 0; + return mvccCompare(r1, r2); + } + + /** + * @param io IO. + * @param pageAddr Page address. + * @param idx Item index. + * @param r2 Search row. + * @return Comparison result. + */ + private int mvccCompare(H2RowLinkIO io, long pageAddr, int idx, GridH2SearchRow r2) { + if (!mvccEnabled || r2.indexSearchRow()) + return 0; + + long crd = io.getMvccCoordinatorVersion(pageAddr, idx); + long cntr = io.getMvccCounter(pageAddr, idx); + int opCntr = io.getMvccOperationCounter(pageAddr, idx); + + assert MvccUtils.mvccVersionIsValid(crd, cntr, opCntr); + + return -MvccUtils.compare(crd, cntr, opCntr, r2); // descending order + } + + /** + * @param r1 First row. + * @param r2 Second row. + * @return Comparison result. + */ + private int mvccCompare(GridH2SearchRow r1, GridH2SearchRow r2) { + if (!mvccEnabled || r2.indexSearchRow()) + return 0; + + long crdVer1 = r1.mvccCoordinatorVersion(); + long crdVer2 = r2.mvccCoordinatorVersion(); + + int c = -Long.compare(crdVer1, crdVer2); + + if (c != 0) + return c; + + return -Long.compare(r1.mvccCounter(), r2.mvccCounter()); } /** @@ -306,4 +367,9 @@ public int compareRows(SearchRow r1, SearchRow r2) { * @return Comparison result. */ public abstract int compareValues(Value v1, Value v2); + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(H2Tree.class, this, "super", super.toString()); + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java new file mode 100644 index 0000000000000..e583546fe8d24 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java @@ -0,0 +1,109 @@ +/* + * 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.database; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; +import org.apache.ignite.internal.transactions.IgniteTxMvccVersionCheckedException; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; + +import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.isVisible; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccVersionIsValid; + +/** + * + */ +public class H2TreeFilterClosure implements H2Tree.TreeRowClosure { + /** */ + private final MvccSnapshot mvccSnapshot; + + /** */ + private final IndexingQueryCacheFilter filter; + + /** */ + private final GridCacheContext cctx; + + /** + * @param filter Cache filter. + * @param mvccSnapshot MVCC snapshot. + * @param cctx Cache context. + */ + public H2TreeFilterClosure(IndexingQueryCacheFilter filter, MvccSnapshot mvccSnapshot, GridCacheContext cctx) { + assert (filter != null || mvccSnapshot != null) && cctx != null ; + + this.filter = filter; + this.mvccSnapshot = mvccSnapshot; + this.cctx = cctx; + } + + /** {@inheritDoc} */ + @Override public boolean apply(BPlusTree tree, BPlusIO io, + long pageAddr, int idx) throws IgniteCheckedException { + return (filter == null || applyFilter((H2RowLinkIO)io, pageAddr, idx)) + && (mvccSnapshot == null || applyMvcc((H2RowLinkIO)io, pageAddr, idx)); + } + + /** + * @param io Row IO. + * @param pageAddr Page address. + * @param idx Item index. + * @return {@code True} if row passes the filter. + */ + private boolean applyFilter(H2RowLinkIO io, long pageAddr, int idx) { + assert filter != null; + + return filter.applyPartition(PageIdUtils.partId(pageId(io.getLink(pageAddr, idx)))); + } + + /** + * @param io Row IO. + * @param pageAddr Page address. + * @param idx Item index. + * @return {@code True} if row passes the filter. + */ + private boolean applyMvcc(H2RowLinkIO io, long pageAddr, int idx) throws IgniteCheckedException { + assert io.storeMvccInfo() : io; + + long rowCrdVer = io.getMvccCoordinatorVersion(pageAddr, idx); + long rowCntr = io.getMvccCounter(pageAddr, idx); + int rowOpCntr = io.getMvccOperationCounter(pageAddr, idx); + + assert mvccVersionIsValid(rowCrdVer, rowCntr, rowOpCntr); + + try { + return isVisible(cctx, mvccSnapshot, rowCrdVer, rowCntr, rowOpCntr, io.getLink(pageAddr, idx)); + } + catch (IgniteTxMvccVersionCheckedException ignored) { + return false; // The row is going to be removed. + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(H2TreeFilterClosure.class, this); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java index 393ca3b5352b8..ab6f42a44f7e6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java @@ -20,36 +20,34 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; -import java.util.NoSuchElementException; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.RootPage; import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.h2.H2Cursor; import org.apache.ignite.internal.processors.query.h2.H2RowCache; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Cursor; import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; -import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; import org.apache.ignite.internal.util.IgniteTree; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; +import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.h2.engine.Session; import org.h2.index.Cursor; import org.h2.index.IndexType; import org.h2.index.SingleRowCursor; import org.h2.message.DbException; -import org.h2.result.Row; import org.h2.result.SearchRow; import org.h2.result.SortOrder; import org.h2.table.Column; @@ -139,6 +137,7 @@ public H2TreeIndex( cols, inlineIdxs, computeInlineSize(inlineIdxs, inlineSize), + cctx.mvccEnabled(), rowCache, cctx.kernalContext().failure()) { @Override public int compareValues(Value v1, Value v2) { @@ -191,21 +190,22 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { /** {@inheritDoc} */ @Override public Cursor find(Session ses, SearchRow lower, SearchRow upper) { try { - IndexingQueryCacheFilter filter = partitionFilter(threadLocalFilter()); + assert lower == null || lower instanceof GridH2SearchRow : lower; + assert upper == null || upper instanceof GridH2SearchRow : upper; int seg = threadLocalSegment(); H2Tree tree = treeForRead(seg); - if (indexType.isPrimaryKey() && lower != null && upper != null && tree.compareRows(lower, upper) == 0) { - GridH2Row row = tree.findOne(lower, filter); + if (!cctx.mvccEnabled() && indexType.isPrimaryKey() && lower != null && upper != null && + tree.compareRows((GridH2SearchRow)lower, (GridH2SearchRow)upper) == 0) { + GridH2Row row = tree.findOne((GridH2SearchRow)lower, filter(GridH2QueryContext.get()), null); - return (row == null) ? EMPTY_CURSOR : new SingleRowCursor(row); + return (row == null) ? GridH2Cursor.EMPTY : new SingleRowCursor(row); } else { - GridCursor cursor = tree.find(lower, upper, filter); - - return new H2Cursor(cursor); + return new H2Cursor(tree.find((GridH2SearchRow)lower, + (GridH2SearchRow)upper, filter(GridH2QueryContext.get()), null)); } } catch (IgniteCheckedException e) { @@ -257,6 +257,8 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { /** {@inheritDoc} */ @Override public GridH2Row remove(SearchRow row) { + assert row instanceof GridH2SearchRow : row; + try { InlineIndexHelper.setCurrentInlineIndexes(inlineIdxs); @@ -266,7 +268,7 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { assert cctx.shared().database().checkpointLockIsHeldByThread(); - return tree.remove(row); + return tree.remove((GridH2SearchRow)row); } catch (IgniteCheckedException e) { throw DbException.convert(e); @@ -279,6 +281,8 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { /** {@inheritDoc} */ @Override public boolean removex(SearchRow row) { try { + assert row instanceof GridH2SearchRow : row; + InlineIndexHelper.setCurrentInlineIndexes(inlineIdxs); int seg = segmentForRow(row); @@ -287,7 +291,7 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { assert cctx.shared().database().checkpointLockIsHeldByThread(); - return tree.removex(row); + return tree.removex((GridH2SearchRow)row); } catch (IgniteCheckedException e) { throw DbException.convert(e); @@ -315,9 +319,9 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { H2Tree tree = treeForRead(seg); - BPlusTree.TreeRowClosure filter = filterClosure(); + GridH2QueryContext qctx = GridH2QueryContext.get(); - return tree.size(filter); + return tree.size(filter(qctx)); } catch (IgniteCheckedException e) { throw DbException.convert(e); @@ -337,13 +341,10 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { /** {@inheritDoc} */ @Override public Cursor findFirstOrLast(Session session, boolean b) { try { - int seg = threadLocalSegment(); + H2Tree tree = treeForRead(threadLocalSegment()); + GridH2QueryContext qctx = GridH2QueryContext.get(); - H2Tree tree = treeForRead(seg); - - GridH2Row row = b ? tree.findFirst(): tree.findLast(); - - return new SingleRowCursor(row); + return new SingleRowCursor(b ? tree.findFirst(filter(qctx)): tree.findLast(filter(qctx))); } catch (IgniteCheckedException e) { throw DbException.convert(e); @@ -382,16 +383,13 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { @Override protected H2Cursor doFind0( IgniteTree t, @Nullable SearchRow first, - boolean includeFirst, @Nullable SearchRow last, - IndexingQueryFilter filter) { + BPlusTree.TreeRowClosure filter) { try { - IndexingQueryCacheFilter pf = partitionFilter(filter); - - GridCursor range = t.find(first, last, pf); + GridCursor range = ((BPlusTree)t).find(first, last, filter, null); if (range == null) - range = GridH2IndexBase.EMPTY_CURSOR; + range = EMPTY_CURSOR; return new H2Cursor(range); } @@ -400,6 +398,26 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { } } + /** {@inheritDoc} */ + @Override protected BPlusTree.TreeRowClosure filter(GridH2QueryContext qctx) { + if (qctx == null) { + assert !cctx.mvccEnabled(); + + return null; + } + + IndexingQueryFilter f = qctx.filter(); + IndexingQueryCacheFilter p = f == null ? null : f.forCache(getTable().cacheName()); + MvccSnapshot v = qctx.mvccSnapshot(); + + assert !cctx.mvccEnabled() || v != null; + + if(p == null && v == null) + return null; + + return new H2TreeFilterClosure(p, v, cctx); + } + /** * @param inlineIdxs Inline index helpers. * @param cfgInlineSize Inline size from cache config. @@ -457,63 +475,6 @@ private void dropMetaPage(String name, int segIdx) throws IgniteCheckedException cctx.offheap().dropRootPageForIndex(cctx.cacheId(), name + "%" + segIdx); } - /** - * Returns a filter which returns true for entries belonging to a particular partition. - * - * @param qryFilter Factory that creates a predicate for filtering entries for a particular cache. - * @return The filter or null if the filter is not needed (e.g., if the cache is not partitioned). - */ - @Nullable private IndexingQueryCacheFilter partitionFilter(@Nullable IndexingQueryFilter qryFilter) { - if (qryFilter == null) - return null; - - String cacheName = getTable().cacheName(); - - return qryFilter.forCache(cacheName); - } - - /** - * An adapter from {@link IndexingQueryCacheFilter} to {@link BPlusTree.TreeRowClosure} which - * filters entries that belong to the current partition. - */ - private static class PartitionFilterTreeRowClosure implements BPlusTree.TreeRowClosure { - /** Filter. */ - private final IndexingQueryCacheFilter filter; - - /** - * Creates a {@link BPlusTree.TreeRowClosure} adapter based on the given partition filter. - * - * @param filter The partition filter. - */ - public PartitionFilterTreeRowClosure(IndexingQueryCacheFilter filter) { - this.filter = filter; - } - - /** {@inheritDoc} */ - @Override public boolean apply(BPlusTree tree, - BPlusIO io, long pageAddr, int idx) throws IgniteCheckedException { - - H2RowLinkIO h2io = (H2RowLinkIO)io; - - return filter.applyPartition( - PageIdUtils.partId( - PageIdUtils.pageId( - h2io.getLink(pageAddr, idx)))); - } - } - - /** - * Returns a filter to apply to rows in the current index to obtain only the - * ones owned by the this cache. - * - * @return The filter, which returns true for rows owned by this cache. - */ - @Nullable private BPlusTree.TreeRowClosure filterClosure() { - final IndexingQueryCacheFilter filter = partitionFilter(threadLocalFilter()); - - return filter != null ? new PartitionFilterTreeRowClosure(filter) : null; - } - /** {@inheritDoc} */ @Override public void refreshColumnIds() { super.refreshColumnIds(); @@ -528,29 +489,4 @@ public PartitionFilterTreeRowClosure(IndexingQueryCacheFilter filter) { for (int pos = 0; pos < inlineHelpers.size(); ++pos) inlineIdxs.set(pos, inlineHelpers.get(pos)); } - - /** - * Empty cursor. - */ - public static final Cursor EMPTY_CURSOR = new Cursor() { - /** {@inheritDoc} */ - @Override public Row get() { - throw DbException.convert(new NoSuchElementException("Empty cursor")); - } - - /** {@inheritDoc} */ - @Override public SearchRow getSearchRow() { - throw DbException.convert(new NoSuchElementException("Empty cursor")); - } - - /** {@inheritDoc} */ - @Override public boolean next() { - return false; - } - - /** {@inheritDoc} */ - @Override public boolean previous() { - return false; - } - }; } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasInnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasInnerIO.java new file mode 100644 index 0000000000000..fbca9176d48c6 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasInnerIO.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2.database.io; + +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.processors.query.h2.database.H2Tree; +import org.apache.ignite.internal.processors.query.h2.database.InlineIndexHelper; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; + +/** + * Inner page for H2 row references. + */ +public abstract class AbstractH2ExtrasInnerIO extends BPlusInnerIO implements H2RowLinkIO { + /** Payload size. */ + protected final int payloadSize; + + /** */ + public static void register() { + register(false); + + register(true); + } + + /** + * @param mvcc Mvcc flag. + */ + private static void register(boolean mvcc) { + short type = mvcc ? PageIO.T_H2_EX_REF_MVCC_INNER_START : PageIO.T_H2_EX_REF_INNER_START; + + for (short payload = 1; payload <= PageIO.MAX_PAYLOAD_SIZE; payload++) { + IOVersions io = + getVersions((short)(type + payload - 1), payload, mvcc); + + PageIO.registerH2ExtraInner(io, mvcc); + } + } + + /** + * @param payload Payload size. + * @param mvccEnabled Mvcc flag. + * @return IOVersions for given payload. + */ + @SuppressWarnings("unchecked") + public static IOVersions> getVersions(int payload, boolean mvccEnabled) { + assert payload >= 0 && payload <= PageIO.MAX_PAYLOAD_SIZE; + + if (payload == 0) + return mvccEnabled ? H2MvccInnerIO.VERSIONS : H2InnerIO.VERSIONS; + else + return (IOVersions>)PageIO.getInnerVersions((short)(payload - 1), mvccEnabled); + } + + /** + * @param type Type. + * @param payload Payload size. + * @param mvcc Mvcc flag. + * @return Instance of IO versions. + */ + private static IOVersions getVersions(short type, short payload, boolean mvcc) { + return new IOVersions<>(mvcc ? new H2MvccExtrasInnerIO(type, 1, payload) : new H2ExtrasInnerIO(type, 1, payload)); + } + + /** + * @param type Page type. + * @param ver Page format version. + * @param itemSize Item size. + * @param payloadSize Payload size. + */ + AbstractH2ExtrasInnerIO(short type, int ver, int itemSize, int payloadSize) { + super(type, ver, true, itemSize + payloadSize); + + this.payloadSize = payloadSize; + } + + /** {@inheritDoc} */ + @SuppressWarnings("ForLoopReplaceableByForEach") + @Override public final void storeByOffset(long pageAddr, int off, GridH2SearchRow row) { + GridH2Row row0 = (GridH2Row)row; + + assert row0.link() != 0 : row0; + + List inlineIdxs = InlineIndexHelper.getCurrentInlineIndexes(); + + assert inlineIdxs != null : "no inline index helpers"; + + + int fieldOff = 0; + + for (int i = 0; i < inlineIdxs.size(); i++) { + InlineIndexHelper idx = inlineIdxs.get(i); + + int size = idx.put(pageAddr, off + fieldOff, row.getValue(idx.columnIndex()), payloadSize - fieldOff); + + if (size == 0) + break; + + fieldOff += size; + } + + H2IOUtils.storeRow(row0, pageAddr, off + payloadSize, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public final GridH2SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) + throws IgniteCheckedException { + long link = getLink(pageAddr, idx); + + assert link != 0; + + if (storeMvccInfo()) { + long mvccCrdVer = getMvccCoordinatorVersion(pageAddr, idx); + long mvccCntr = getMvccCounter(pageAddr, idx); + int mvccOpCntr = getMvccOperationCounter(pageAddr, idx); + + return ((H2Tree)tree).createRowFromLink(link, mvccCrdVer, mvccCntr, mvccOpCntr); + } + + return ((H2Tree)tree).createRowFromLink(link); + } + + /** {@inheritDoc} */ + @Override public final void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { + int srcOff = srcIo.offset(srcIdx); + + byte[] payload = PageUtils.getBytes(srcPageAddr, srcOff, payloadSize); + long link = PageUtils.getLong(srcPageAddr, srcOff + payloadSize); + + assert link != 0; + + int dstOff = offset(dstIdx); + + PageUtils.putBytes(dstPageAddr, dstOff, payload); + + H2IOUtils.store(dstPageAddr, dstOff + payloadSize, srcIo, srcPageAddr, srcIdx, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public final long getLink(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + payloadSize); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasLeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasLeafIO.java new file mode 100644 index 0000000000000..913279553b67c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2ExtrasLeafIO.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.internal.processors.query.h2.database.io; + +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.processors.query.h2.database.H2Tree; +import org.apache.ignite.internal.processors.query.h2.database.InlineIndexHelper; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; + +/** + * Leaf page for H2 row references. + */ +public abstract class AbstractH2ExtrasLeafIO extends BPlusLeafIO implements H2RowLinkIO { + /** Payload size. */ + protected final int payloadSize; + + /** */ + public static void register() { + register(false); + + register(true); + } + + /** + * @param mvcc Mvcc flag. + */ + private static void register(boolean mvcc) { + short type = mvcc ? PageIO.T_H2_EX_REF_MVCC_LEAF_START : PageIO.T_H2_EX_REF_LEAF_START; + + for (short payload = 1; payload <= PageIO.MAX_PAYLOAD_SIZE; payload++) { + IOVersions io = + getVersions((short)(type + payload - 1), payload, mvcc); + + PageIO.registerH2ExtraLeaf(io, mvcc); + } + } + + /** + * @param payload Payload size. + * @param mvccEnabled Mvcc flag. + * @return IOVersions for given payload. + */ + @SuppressWarnings("unchecked") + public static IOVersions> getVersions(int payload, boolean mvccEnabled) { + assert payload >= 0 && payload <= PageIO.MAX_PAYLOAD_SIZE; + + if (payload == 0) + return mvccEnabled ? H2MvccLeafIO.VERSIONS : H2LeafIO.VERSIONS; + else + return (IOVersions>)PageIO.getLeafVersions((short)(payload - 1), mvccEnabled); + } + + /** + * @param type Type. + * @param payload Payload size. + * @param mvcc Mvcc flag. + * @return Versions. + */ + private static IOVersions getVersions(short type, short payload, boolean mvcc) { + return new IOVersions<>(mvcc ? new H2MvccExtrasLeafIO(type, 1, payload) : new H2ExtrasLeafIO(type, 1, payload)); + } + + /** + * @param type Page type. + * @param ver Page format version. + * @param itemSize Item size. + * @param payloadSize Payload size. + */ + AbstractH2ExtrasLeafIO(short type, int ver, int itemSize, int payloadSize) { + super(type, ver, itemSize + payloadSize); + + this.payloadSize = payloadSize; + } + + /** {@inheritDoc} */ + @SuppressWarnings("ForLoopReplaceableByForEach") + @Override public final void storeByOffset(long pageAddr, int off, GridH2SearchRow row) { + GridH2Row row0 = (GridH2Row)row; + + assert row0.link() != 0; + + List inlineIdxs = InlineIndexHelper.getCurrentInlineIndexes(); + + assert inlineIdxs != null : "no inline index helpers"; + + int fieldOff = 0; + + for (int i = 0; i < inlineIdxs.size(); i++) { + InlineIndexHelper idx = inlineIdxs.get(i); + + int size = idx.put(pageAddr, off + fieldOff, row.getValue(idx.columnIndex()), payloadSize - fieldOff); + + if (size == 0) + break; + + fieldOff += size; + } + + H2IOUtils.storeRow(row0, pageAddr, off + payloadSize, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public final void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { + int srcOff = srcIo.offset(srcIdx); + + byte[] payload = PageUtils.getBytes(srcPageAddr, srcOff, payloadSize); + long link = PageUtils.getLong(srcPageAddr, srcOff + payloadSize); + + assert link != 0; + + int dstOff = offset(dstIdx); + + PageUtils.putBytes(dstPageAddr, dstOff, payload); + + H2IOUtils.store(dstPageAddr, dstOff + payloadSize, srcIo, srcPageAddr, srcIdx, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public final GridH2SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) + throws IgniteCheckedException { + long link = getLink(pageAddr, idx); + + if (storeMvccInfo()) { + long mvccCrdVer = getMvccCoordinatorVersion(pageAddr, idx); + long mvccCntr = getMvccCounter(pageAddr, idx); + int mvccOpCntr = getMvccOperationCounter(pageAddr, idx); + + return ((H2Tree)tree).createRowFromLink(link, mvccCrdVer, mvccCntr, mvccOpCntr); + } + + return ((H2Tree)tree).createRowFromLink(link); + } + + /** {@inheritDoc} */ + @Override public final long getLink(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + payloadSize); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2InnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2InnerIO.java new file mode 100644 index 0000000000000..d1d569e98c385 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2InnerIO.java @@ -0,0 +1,74 @@ +/* + * 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.database.io; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO; +import org.apache.ignite.internal.processors.query.h2.database.H2Tree; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; + +/** + * Inner page for H2 row references. + */ +public abstract class AbstractH2InnerIO extends BPlusInnerIO implements H2RowLinkIO { + /** + * @param type Page type. + * @param ver Page format version. + * @param itemSize Single item size on page. + */ + AbstractH2InnerIO(int type, int ver, int itemSize) { + super(type, ver, true, itemSize); + } + + /** {@inheritDoc} */ + @Override public void storeByOffset(long pageAddr, int off, GridH2SearchRow row) { + GridH2Row row0 = (GridH2Row)row; + + H2IOUtils.storeRow(row0, pageAddr, off, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public GridH2SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) + throws IgniteCheckedException { + long link = getLink(pageAddr, idx); + + if (storeMvccInfo()) { + long mvccCrdVer = getMvccCoordinatorVersion(pageAddr, idx); + long mvccCntr = getMvccCounter(pageAddr, idx); + int mvccOpCntr = getMvccOperationCounter(pageAddr, idx); + + return ((H2Tree)tree).createRowFromLink(link, mvccCrdVer, mvccCntr, mvccOpCntr); + } + + return ((H2Tree)tree).createRowFromLink(link); + } + + /** {@inheritDoc} */ + @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { + H2IOUtils.store(dstPageAddr, offset(dstIdx), srcIo, srcPageAddr, srcIdx, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public long getLink(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx)); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2LeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2LeafIO.java new file mode 100644 index 0000000000000..07f114ffbfc23 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2LeafIO.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.processors.query.h2.database.io; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO; +import org.apache.ignite.internal.processors.query.h2.database.H2Tree; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; + +/** + * Leaf page for H2 row references. + */ +public abstract class AbstractH2LeafIO extends BPlusLeafIO implements H2RowLinkIO { + /** + * @param type Page type. + * @param ver Page format version. + * @param itemSize Single item size on page. + */ + AbstractH2LeafIO(int type, int ver, int itemSize) { + super(type, ver, itemSize); + } + + /** {@inheritDoc} */ + @Override public final void storeByOffset(long pageAddr, int off, GridH2SearchRow row) { + GridH2Row row0 = (GridH2Row)row; + + H2IOUtils.storeRow(row0, pageAddr, off, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public final void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { + assert srcIo == this; + + H2IOUtils.store(dstPageAddr, offset(dstIdx), srcIo, srcPageAddr, srcIdx, storeMvccInfo()); + } + + /** {@inheritDoc} */ + @Override public final GridH2SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) + throws IgniteCheckedException { + long link = getLink(pageAddr, idx); + + if (storeMvccInfo()) { + long mvccCrdVer = getMvccCoordinatorVersion(pageAddr, idx); + long mvccCntr = getMvccCounter(pageAddr, idx); + int mvccOpCntr = getMvccOperationCounter(pageAddr, idx); + + return ((H2Tree)tree).createRowFromLink(link, mvccCrdVer, mvccCntr, mvccOpCntr); + } + + return ((H2Tree)tree).createRowFromLink(link); + } + + /** {@inheritDoc} */ + @Override public long getLink(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx)); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java index 294492debf1b7..8dc8c96e83568 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java @@ -17,120 +17,17 @@ package org.apache.ignite.internal.processors.query.h2.database.io; -import java.util.List; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.pagemem.PageUtils; -import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; -import org.apache.ignite.internal.processors.query.h2.database.H2Tree; -import org.apache.ignite.internal.processors.query.h2.database.InlineIndexHelper; -import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; -import org.h2.result.SearchRow; - /** * Inner page for H2 row references. */ -public class H2ExtrasInnerIO extends BPlusInnerIO implements H2RowLinkIO { - /** Payload size. */ - private final int payloadSize; - - /** */ - public static void register() { - for (short payload = 1; payload <= PageIO.MAX_PAYLOAD_SIZE; payload++) - PageIO.registerH2ExtraInner(getVersions((short)(PageIO.T_H2_EX_REF_INNER_START + payload - 1), payload)); - } - - /** - * @param payload Payload size. - * @return IOVersions for given payload. - */ - @SuppressWarnings("unchecked") - public static IOVersions> getVersions(int payload) { - assert payload >= 0 && payload <= PageIO.MAX_PAYLOAD_SIZE; - - if (payload == 0) - return H2InnerIO.VERSIONS; - else - return (IOVersions>)PageIO.getInnerVersions((short)(payload - 1)); - } - - /** - * @param type Type. - * @param payload Payload size. - * @return Instance of IO versions. - */ - private static IOVersions getVersions(short type, short payload) { - return new IOVersions<>(new H2ExtrasInnerIO(type, 1, payload)); - } - +public class H2ExtrasInnerIO extends AbstractH2ExtrasInnerIO implements H2RowLinkIO { /** * @param type Page type. * @param ver Page format version. * @param payloadSize Payload size. */ - private H2ExtrasInnerIO(short type, int ver, int payloadSize) { - super(type, ver, true, 8 + payloadSize); - this.payloadSize = payloadSize; - } - - /** {@inheritDoc} */ - @SuppressWarnings("ForLoopReplaceableByForEach") - @Override public void storeByOffset(long pageAddr, int off, SearchRow row) { - GridH2Row row0 = (GridH2Row)row; - - assert row0.link() != 0 : row0; - - List inlineIdxs = InlineIndexHelper.getCurrentInlineIndexes(); - - assert inlineIdxs != null : "no inline index helpers"; - - - int fieldOff = 0; - - for (int i = 0; i < inlineIdxs.size(); i++) { - InlineIndexHelper idx = inlineIdxs.get(i); - - int size = idx.put(pageAddr, off + fieldOff, row.getValue(idx.columnIndex()), payloadSize - fieldOff); - - if (size == 0) - break; - - fieldOff += size; - } - - PageUtils.putLong(pageAddr, off + payloadSize, row0.link()); - } - - /** {@inheritDoc} */ - @Override public SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) - throws IgniteCheckedException { - long link = getLink(pageAddr, idx); - - assert link != 0; - - return ((H2Tree)tree).createRowFromLink(link); - } - - /** {@inheritDoc} */ - @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { - int srcOff = srcIo.offset(srcIdx); - - byte[] payload = PageUtils.getBytes(srcPageAddr, srcOff, payloadSize); - long link = PageUtils.getLong(srcPageAddr, srcOff + payloadSize); - - assert link != 0; - - int dstOff = offset(dstIdx); - - PageUtils.putBytes(dstPageAddr, dstOff, payload); - PageUtils.putLong(dstPageAddr, dstOff + payloadSize, link); - } - - /** {@inheritDoc} */ - @Override public long getLink(long pageAddr, int idx) { - return PageUtils.getLong(pageAddr, offset(idx) + payloadSize); + H2ExtrasInnerIO(short type, int ver, int payloadSize) { + super(type, ver, 8, payloadSize); } } + diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java index 47702959bc879..085f98bd9817e 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java @@ -17,117 +17,16 @@ package org.apache.ignite.internal.processors.query.h2.database.io; -import java.util.List; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.pagemem.PageUtils; -import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; -import org.apache.ignite.internal.processors.query.h2.database.H2Tree; -import org.apache.ignite.internal.processors.query.h2.database.InlineIndexHelper; -import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; -import org.h2.result.SearchRow; - /** * Leaf page for H2 row references. */ -public class H2ExtrasLeafIO extends BPlusLeafIO implements H2RowLinkIO { - /** Payload size. */ - private final int payloadSize; - - /** */ - public static void register() { - for (short payload = 1; payload <= PageIO.MAX_PAYLOAD_SIZE; payload++) - PageIO.registerH2ExtraLeaf(getVersions((short)(PageIO.T_H2_EX_REF_LEAF_START + payload - 1), payload)); - } - - /** - * @param payload Payload size. - * @return IOVersions for given payload. - */ - @SuppressWarnings("unchecked") - public static IOVersions> getVersions(int payload) { - assert payload >= 0 && payload <= PageIO.MAX_PAYLOAD_SIZE; - - if (payload == 0) - return H2LeafIO.VERSIONS; - else - return (IOVersions>)PageIO.getLeafVersions((short)(payload - 1)); - } - - /** - * @param type Type. - * @param payload Payload size. - * @return Versions. - */ - private static IOVersions getVersions(short type, short payload) { - return new IOVersions<>(new H2ExtrasLeafIO(type, 1, payload)); - } - +public class H2ExtrasLeafIO extends AbstractH2ExtrasLeafIO { /** * @param type Page type. * @param ver Page format version. * @param payloadSize Payload size. */ - private H2ExtrasLeafIO(short type, int ver, int payloadSize) { - super(type, ver, 8 + payloadSize); - this.payloadSize = payloadSize; - } - - /** {@inheritDoc} */ - @SuppressWarnings("ForLoopReplaceableByForEach") - @Override public void storeByOffset(long pageAddr, int off, SearchRow row) { - GridH2Row row0 = (GridH2Row)row; - - assert row0.link() != 0; - - List inlineIdxs = InlineIndexHelper.getCurrentInlineIndexes(); - - assert inlineIdxs != null : "no inline index helpers"; - - int fieldOff = 0; - - for (int i = 0; i < inlineIdxs.size(); i++) { - InlineIndexHelper idx = inlineIdxs.get(i); - - int size = idx.put(pageAddr, off + fieldOff, row.getValue(idx.columnIndex()), payloadSize - fieldOff); - - if (size == 0) - break; - - fieldOff += size; - } - - PageUtils.putLong(pageAddr, off + payloadSize, row0.link()); - } - - /** {@inheritDoc} */ - @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { - int srcOff = srcIo.offset(srcIdx); - - byte[] payload = PageUtils.getBytes(srcPageAddr, srcOff, payloadSize); - long link = PageUtils.getLong(srcPageAddr, srcOff + payloadSize); - - assert link != 0; - - int dstOff = offset(dstIdx); - - PageUtils.putBytes(dstPageAddr, dstOff, payload); - PageUtils.putLong(dstPageAddr, dstOff + payloadSize, link); - } - - /** {@inheritDoc} */ - @Override public SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) - throws IgniteCheckedException { - long link = getLink(pageAddr, idx); - - return ((H2Tree)tree).createRowFromLink(link); - } - - /** {@inheritDoc} */ - @Override public long getLink(long pageAddr, int idx) { - return PageUtils.getLong(pageAddr, offset(idx) + payloadSize); + H2ExtrasLeafIO(short type, int ver, int payloadSize) { + super(type, ver, 8, payloadSize); } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2IOUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2IOUtils.java new file mode 100644 index 0000000000000..b6bda36358d9c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2IOUtils.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2.database.io; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2SearchRow; + +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccVersionIsValid; + +/** + * + */ +class H2IOUtils { + /** + * + */ + private H2IOUtils() {} + + /** + * @param row Row. + * @param pageAddr Page address. + * @param off Offset. + * @param storeMvcc {@code True} to store mvcc data. + */ + static void storeRow(GridH2Row row, long pageAddr, int off, boolean storeMvcc) { + assert row.link() != 0; + + PageUtils.putLong(pageAddr, off, row.link()); + + if (storeMvcc) { + long mvccCrdVer = row.mvccCoordinatorVersion(); + long mvccCntr = row.mvccCounter(); + int mvccOpCntr = row.mvccOperationCounter(); + + assert MvccUtils.mvccVersionIsValid(mvccCrdVer, mvccCntr, mvccOpCntr); + + PageUtils.putLong(pageAddr, off + 8, mvccCrdVer); + PageUtils.putLong(pageAddr, off + 16, mvccCntr); + PageUtils.putInt(pageAddr, off + 24, mvccOpCntr); + } + } + + /** + * @param dstPageAddr Destination page address. + * @param dstOff Destination page offset. + * @param srcIo Source IO. + * @param srcPageAddr Source page address. + * @param srcIdx Source index. + * @param storeMvcc {@code True} to store mvcc data. + */ + static void store(long dstPageAddr, + int dstOff, + BPlusIO srcIo, + long srcPageAddr, + int srcIdx, + boolean storeMvcc) + { + H2RowLinkIO rowIo = (H2RowLinkIO)srcIo; + + long link = rowIo.getLink(srcPageAddr, srcIdx); + + PageUtils.putLong(dstPageAddr, dstOff, link); + + if (storeMvcc) { + long mvccCrdVer = rowIo.getMvccCoordinatorVersion(srcPageAddr, srcIdx); + long mvccCntr = rowIo.getMvccCounter(srcPageAddr, srcIdx); + int mvccOpCntr = rowIo.getMvccOperationCounter(srcPageAddr, srcIdx); + + assert MvccUtils.mvccVersionIsValid(mvccCrdVer, mvccCntr, mvccOpCntr); + + PageUtils.putLong(dstPageAddr, dstOff + 8, mvccCrdVer); + PageUtils.putLong(dstPageAddr, dstOff + 16, mvccCntr); + PageUtils.putInt(dstPageAddr, dstOff + 24, mvccOpCntr); + } + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java index cf37bb74f2170..9baff7a05b312 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java @@ -17,20 +17,12 @@ package org.apache.ignite.internal.processors.query.h2.database.io; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.pagemem.PageUtils; -import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; -import org.apache.ignite.internal.processors.query.h2.database.H2Tree; -import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; -import org.h2.result.SearchRow; /** * Inner page for H2 row references. */ -public class H2InnerIO extends BPlusInnerIO implements H2RowLinkIO { +public class H2InnerIO extends AbstractH2InnerIO { /** */ public static final IOVersions VERSIONS = new IOVersions<>( new H2InnerIO(1) @@ -40,35 +32,6 @@ public class H2InnerIO extends BPlusInnerIO implements H2RowLinkIO { * @param ver Page format version. */ private H2InnerIO(int ver) { - super(T_H2_REF_INNER, ver, true, 8); - } - - /** {@inheritDoc} */ - @Override public void storeByOffset(long pageAddr, int off, SearchRow row) { - GridH2Row row0 = (GridH2Row)row; - - assert row0.link() != 0; - - PageUtils.putLong(pageAddr, off, row0.link()); - } - - /** {@inheritDoc} */ - @Override public SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) - throws IgniteCheckedException { - long link = getLink(pageAddr, idx); - - return ((H2Tree)tree).createRowFromLink(link); - } - - /** {@inheritDoc} */ - @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { - long link = ((H2RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx); - - PageUtils.putLong(dstPageAddr, offset(dstIdx), link); - } - - /** {@inheritDoc} */ - @Override public long getLink(long pageAddr, int idx) { - return PageUtils.getLong(pageAddr, offset(idx)); + super(T_H2_REF_INNER, ver, 8); } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java index 55a980f62e79a..8954de08e3175 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java @@ -17,20 +17,12 @@ package org.apache.ignite.internal.processors.query.h2.database.io; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.pagemem.PageUtils; -import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; -import org.apache.ignite.internal.processors.query.h2.database.H2Tree; -import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; -import org.h2.result.SearchRow; /** * Leaf page for H2 row references. */ -public class H2LeafIO extends BPlusLeafIO implements H2RowLinkIO { +public class H2LeafIO extends AbstractH2LeafIO { /** */ public static final IOVersions VERSIONS = new IOVersions<>( new H2LeafIO(1) @@ -39,36 +31,7 @@ public class H2LeafIO extends BPlusLeafIO implements H2RowLinkIO { /** * @param ver Page format version. */ - protected H2LeafIO(int ver) { + private H2LeafIO(int ver) { super(T_H2_REF_LEAF, ver, 8); } - - /** {@inheritDoc} */ - @Override public void storeByOffset(long pageAddr, int off, SearchRow row) { - GridH2Row row0 = (GridH2Row)row; - - assert row0.link() != 0; - - PageUtils.putLong(pageAddr, off, row0.link()); - } - - /** {@inheritDoc} */ - @Override public void store(long dstPageAddr, int dstIdx, BPlusIO srcIo, long srcPageAddr, int srcIdx) { - assert srcIo == this; - - PageUtils.putLong(dstPageAddr, offset(dstIdx), getLink(srcPageAddr, srcIdx)); - } - - /** {@inheritDoc} */ - @Override public SearchRow getLookupRow(BPlusTree tree, long pageAddr, int idx) - throws IgniteCheckedException { - long link = getLink(pageAddr, idx); - - return ((H2Tree)tree).createRowFromLink(link); - } - - /** {@inheritDoc} */ - @Override public long getLink(long pageAddr, int idx) { - return PageUtils.getLong(pageAddr, offset(idx)); - } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasInnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasInnerIO.java new file mode 100644 index 0000000000000..ee6dc2a40115c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasInnerIO.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2.database.io; + +import org.apache.ignite.internal.pagemem.PageUtils; + +/** + * + */ +class H2MvccExtrasInnerIO extends AbstractH2ExtrasInnerIO { + /** + * @param type Page type. + * @param ver Page format version. + * @param payloadSize Payload size. + */ + H2MvccExtrasInnerIO(short type, int ver, int payloadSize) { + super(type, ver, 28, payloadSize); + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 8); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 16); + } + + /** {@inheritDoc} */ + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + payloadSize + 24); + } + + /** {@inheritDoc} */ + @Override public boolean storeMvccInfo() { + return true; + } +} + diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasLeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasLeafIO.java new file mode 100644 index 0000000000000..60a15989b9379 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasLeafIO.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2.database.io; + +import org.apache.ignite.internal.pagemem.PageUtils; + +/** + * Leaf page for H2 row references. + */ +class H2MvccExtrasLeafIO extends AbstractH2ExtrasLeafIO { + /** + * @param type Page type. + * @param ver Page format version. + * @param payloadSize Payload size. + */ + H2MvccExtrasLeafIO(short type, int ver, int payloadSize) { + super(type, ver, 28, payloadSize); + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 8); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 16); + } + + /** {@inheritDoc} */ + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + payloadSize + 24); + } + + /** {@inheritDoc} */ + @Override public boolean storeMvccInfo() { + return true; + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccInnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccInnerIO.java new file mode 100644 index 0000000000000..dbfe784ceeb29 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccInnerIO.java @@ -0,0 +1,58 @@ +/* + * 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.database.io; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; + +/** + * Inner page for H2 row references. + */ +public class H2MvccInnerIO extends AbstractH2InnerIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>( + new H2MvccInnerIO(1) + ); + + /** + * @param ver Page format version. + */ + private H2MvccInnerIO(int ver) { + super(T_H2_MVCC_REF_INNER, ver, 28); + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 8); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 16); + } + + /** {@inheritDoc} */ + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 24); + } + + /** {@inheritDoc} */ + @Override public boolean storeMvccInfo() { + return true; + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccLeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccLeafIO.java new file mode 100644 index 0000000000000..c7cd99823852c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccLeafIO.java @@ -0,0 +1,58 @@ +/* + * 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.database.io; + +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions; + +/** + * + */ +public class H2MvccLeafIO extends AbstractH2LeafIO { + /** */ + public static final IOVersions VERSIONS = new IOVersions<>( + new H2MvccLeafIO(1) + ); + + /** + * @param ver Page format version. + */ + private H2MvccLeafIO(int ver) { + super(T_H2_MVCC_REF_LEAF, ver, 28); + } + + /** {@inheritDoc} */ + @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 8); + } + + /** {@inheritDoc} */ + @Override public long getMvccCounter(long pageAddr, int idx) { + return PageUtils.getLong(pageAddr, offset(idx) + 16); + } + + /** {@inheritDoc} */ + @Override public int getMvccOperationCounter(long pageAddr, int idx) { + return PageUtils.getInt(pageAddr, offset(idx) + 24); + } + + /** {@inheritDoc} */ + @Override public boolean storeMvccInfo() { + return true; + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java index ce69197070430..1942069ab2408 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java @@ -27,4 +27,38 @@ public interface H2RowLinkIO { * @return Row link. */ public long getLink(long pageAddr, int idx); + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc coordinator version. + */ + public default long getMvccCoordinatorVersion(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc counter. + */ + public default long getMvccCounter(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @param pageAddr Page address. + * @param idx Index. + * @return Mvcc operation counter. + */ + public default int getMvccOperationCounter(long pageAddr, int idx) { + throw new UnsupportedOperationException(); + } + + /** + * @return {@code True} if IO stores mvcc information. + */ + public default boolean storeMvccInfo() { + return false; + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java index 9e88f166566cb..8688c4fbd965a 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java @@ -38,6 +38,8 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.query.GridQueryProperty; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; @@ -125,6 +127,8 @@ public FieldsQueryCursor> runDdlStatement(String sql, SqlCommand cmd) { try { isDdlOnSchemaSupported(cmd.schemaName()); + finishActiveTxIfNecessary(); + if (cmd instanceof SqlCreateIndexCommand) { SqlCreateIndexCommand cmd0 = (SqlCreateIndexCommand)cmd; @@ -260,6 +264,8 @@ public FieldsQueryCursor> runDdlStatement(String sql, Prepared prepared) IgniteInternalFuture fut = null; try { + finishActiveTxIfNecessary(); + GridSqlStatement stmt0 = new GridSqlQueryParser(false).parse(prepared); if (stmt0 instanceof GridSqlCreateIndex) { @@ -442,6 +448,10 @@ else if (stmt0 instanceof GridSqlAlterTableDropColumn) { else { assert tbl.rowDescriptor() != null; + if (tbl.cache().mvccEnabled()) + throw new IgniteSQLException("Cannot drop column(s) with enabled MVCC. " + + "Operation is unsupported at the moment.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + if (QueryUtils.isSqlType(tbl.rowDescriptor().type().valueClass())) throw new SchemaOperationException("Cannot drop column(s) because table was created " + "with " + PARAM_WRAP_VALUE + "=false option."); @@ -569,6 +579,23 @@ private static void isDdlSupported(GridH2Table tbl) { IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } + /** + * Commits active transaction if exists. + * + * @throws IgniteCheckedException If failed. + */ + private void finishActiveTxIfNecessary() throws IgniteCheckedException { + try (GridNearTxLocal tx = MvccUtils.tx(ctx)) { + if (tx == null) + return; + + if (!tx.isRollbackOnly()) + tx.commit(); + else + tx.rollback(); + } + } + /** * @return {@link IgniteSQLException} with the message same as of {@code this}'s and */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java index 161ff4a471b22..ebf58485e7410 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java @@ -98,8 +98,6 @@ public static GridSqlQuery selectForInsertOrMerge(GridSqlColumn[] cols, List processRowForUpdate(List row) throws Ignite return new T3<>(key, oldVal, newVal); } + /** + * @return {@code True} if DML can be fast processed. + */ + public boolean fastResult() { + return fastUpdate != null; + } + /** * Process fast DML operation if possible. * @@ -467,6 +482,48 @@ private void extractArgsValues(Object[] args, List> res, GridH2RowDescri } } + /** + * Create iterator for transaction. + * + * @param idx Indexing. + * @param cur Cursor. + * @return Iterator. + */ + public UpdateSourceIterator iteratorForTransaction(IgniteH2Indexing idx, QueryCursor> cur) { + switch (mode) { + case MERGE: + return new InsertIterator(idx, cur, this, EnlistOperation.UPSERT); + case INSERT: + return new InsertIterator(idx, cur, this, EnlistOperation.INSERT); + case UPDATE: + return new UpdateIterator(idx, cur, this, EnlistOperation.UPDATE); + case DELETE: + return new DeleteIterator(idx, cur, this, EnlistOperation.DELETE); + + default: + throw new IllegalArgumentException(String.valueOf(mode)); + } + } + + /** + * @param updMode Update plan mode. + * @return Operation. + */ + public static EnlistOperation enlistOperation(UpdateMode updMode) { + switch (updMode) { + case INSERT: + return EnlistOperation.INSERT; + case MERGE: + return EnlistOperation.UPSERT; + case UPDATE: + return EnlistOperation.UPDATE; + case DELETE: + return EnlistOperation.DELETE; + default: + throw new IllegalArgumentException(String.valueOf(updMode)); + } + } + /** * @return Update mode. */ @@ -508,4 +565,180 @@ public String selectQuery() { public boolean isLocalSubquery() { return isLocSubqry; } + + /** + * @param args Query parameters. + * @return Iterator. + * @throws IgniteCheckedException If failed. + */ + public IgniteBiTuple getFastRow(Object[] args) throws IgniteCheckedException { + if (fastUpdate != null) + return fastUpdate.getRow(args); + + return null; + } + + /** + * @param row Row. + * @return Resulting entry. + * @throws IgniteCheckedException If failed. + */ + public Object processRowForTx(List row) throws IgniteCheckedException { + switch (mode()) { + case INSERT: + case MERGE: + return processRow(row); + + case UPDATE: { + T3 row0 = processRowForUpdate(row); + + return new IgniteBiTuple<>(row0.get1(), row0.get3()); + } + case DELETE: + return row.get(0); + + default: + throw new UnsupportedOperationException(String.valueOf(mode())); + } + } + + /** + * Abstract iterator. + */ + private abstract static class AbstractIterator extends GridCloseableIteratorAdapterEx + implements UpdateSourceIterator { + /** */ + private final IgniteH2Indexing idx; + + /** */ + private final QueryCursor> cur; + + /** */ + protected final UpdatePlan plan; + + /** */ + private final Iterator> it; + + /** */ + private final EnlistOperation op; + + /** */ + private volatile ThreadLocalObjectPool.Reusable conn; + + /** + * @param idx Indexing. + * @param cur Query cursor. + * @param plan Update plan. + * @param op Operation. + */ + private AbstractIterator(IgniteH2Indexing idx, QueryCursor> cur, UpdatePlan plan, EnlistOperation op) { + this.idx = idx; + this.cur = cur; + this.plan = plan; + this.op = op; + + it = cur.iterator(); + } + + /** {@inheritDoc} */ + @Override public EnlistOperation operation() { + return op; + } + + /** {@inheritDoc} */ + @Override public void beforeDetach() { + ThreadLocalObjectPool.Reusable conn0 = conn = idx.detach(); + + if (isClosed()) + conn0.recycle(); + } + + /** {@inheritDoc} */ + @Override protected void onClose() { + cur.close(); + + ThreadLocalObjectPool.Reusable conn0 = conn; + + if (conn0 != null) + conn0.recycle(); + } + + /** {@inheritDoc} */ + @Override protected Object onNext() throws IgniteCheckedException { + return process(it.next()); + } + + /** {@inheritDoc} */ + @Override protected boolean onHasNext() throws IgniteCheckedException { + return it.hasNext(); + } + + /** */ + protected abstract Object process(List row) throws IgniteCheckedException; + } + + /** */ + private static final class UpdateIterator extends AbstractIterator { + /** */ + private static final long serialVersionUID = -4949035950470324961L; + + /** + * @param idx Indexing. + * @param cur Query cursor. + * @param plan Update plan. + * @param op Operation. + */ + private UpdateIterator(IgniteH2Indexing idx, QueryCursor> cur, UpdatePlan plan, EnlistOperation op) { + super(idx, cur, plan, op); + } + + /** {@inheritDoc} */ + @Override protected Object process(List row) throws IgniteCheckedException { + T3 row0 = plan.processRowForUpdate(row); + + return new IgniteBiTuple<>(row0.get1(), row0.get3()); + } + } + + /** */ + private static final class DeleteIterator extends AbstractIterator { + /** */ + private static final long serialVersionUID = -4949035950470324961L; + + /** + * @param idx Indexing. + * @param cur Query cursor. + * @param plan Update plan. + * @param op Operation. + */ + private DeleteIterator(IgniteH2Indexing idx, QueryCursor> cur, UpdatePlan plan, EnlistOperation op) { + super(idx, cur, plan, op); + } + + /** {@inheritDoc} */ + @Override protected Object process(List row) throws IgniteCheckedException { + return row.get(0); + } + } + + /** */ + private static final class InsertIterator extends AbstractIterator { + /** */ + private static final long serialVersionUID = -4949035950470324961L; + + /** + * @param idx Indexing. + * @param cur Query cursor. + * @param plan Update plan. + * @param op Operation. + */ + private InsertIterator(IgniteH2Indexing idx, QueryCursor> cur, UpdatePlan plan, EnlistOperation op) { + super(idx, cur, plan, op); + } + + /** {@inheritDoc} */ + @Override protected Object process(List row) throws IgniteCheckedException { + return plan.processRow(row); + } + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java index 1079005d42b18..6c103f36fd9e7 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java @@ -41,11 +41,15 @@ import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; +import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAlias; +import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAst; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn; +import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDelete; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlInsert; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlMerge; +import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter; @@ -91,15 +95,48 @@ private UpdatePlanBuilder() { public static UpdatePlan planForStatement(Prepared prepared, boolean loc, IgniteH2Indexing idx, @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQry, @Nullable Integer errKeysPos) throws IgniteCheckedException { - GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared); + assert !prepared.isQuery(); + + GridSqlQueryParser parser = new GridSqlQueryParser(false); + + GridSqlStatement stmt = parser.parse(prepared); + + boolean mvccEnabled = false; + + GridCacheContext cctx = null; + + // check all involved caches + for (Object o : parser.objectsMap().values()) { + if (o instanceof GridSqlInsert) + o = ((GridSqlInsert)o).into(); + else if (o instanceof GridSqlMerge) + o = ((GridSqlMerge)o).into(); + else if (o instanceof GridSqlDelete) + o = ((GridSqlDelete)o).from(); + + if (o instanceof GridSqlAlias) + o = GridSqlAlias.unwrap((GridSqlAst)o); + + if (o instanceof GridSqlTable) { + if (((GridSqlTable)o).dataTable() == null) { // Check for virtual tables. + throw new IgniteSQLException("Operation not supported for table '" + + ((GridSqlTable)o).tableName() + "'", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + } + + if (cctx == null) + mvccEnabled = (cctx = (((GridSqlTable)o).dataTable()).cache()).mvccEnabled(); + else if (((GridSqlTable)o).dataTable().cache().mvccEnabled() != mvccEnabled) + throw new IllegalStateException("Using caches with different mvcc settings in same query is forbidden."); + } + } if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert) - return planForInsert(stmt, loc, idx, conn, fieldsQry); + return planForInsert(stmt, loc, idx, mvccEnabled, conn, fieldsQry); else if (stmt instanceof GridSqlUpdate || stmt instanceof GridSqlDelete) - return planForUpdate(stmt, loc, idx, conn, fieldsQry, errKeysPos); + return planForUpdate(stmt, loc, idx, mvccEnabled, conn, fieldsQry, errKeysPos); else throw new IgniteSQLException("Unsupported operation: " + prepared.getSQL(), - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); } /** @@ -108,6 +145,7 @@ else if (stmt instanceof GridSqlUpdate || stmt instanceof GridSqlDelete) * @param stmt INSERT or MERGE statement. * @param loc Local query flag. * @param idx Indexing. + * @param mvccEnabled Mvcc flag. * @param conn Connection. * @param fieldsQuery Original query. * @return Update plan. @@ -115,8 +153,9 @@ else if (stmt instanceof GridSqlUpdate || stmt instanceof GridSqlDelete) */ @SuppressWarnings("ConstantConditions") private static UpdatePlan planForInsert(GridSqlStatement stmt, boolean loc, IgniteH2Indexing idx, - @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQuery) throws IgniteCheckedException { - GridSqlQuery sel; + boolean mvccEnabled, @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQuery) + throws IgniteCheckedException { + GridSqlQuery sel = null; GridSqlElement target; @@ -140,17 +179,16 @@ private static UpdatePlan planForInsert(GridSqlStatement stmt, boolean loc, Igni GridH2Table h2Tbl = tbl.dataTable(); - if (h2Tbl == null) - throw new IgniteSQLException("Operation not supported for table '" + tbl.tableName() + "'", - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + assert h2Tbl != null; desc = h2Tbl.rowDescriptor(); cols = ins.columns(); - sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query()); - if (sel == null) + if (noQuery(ins.rows())) elRows = ins.rows(); + else + sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query()); isTwoStepSubqry = (ins.query() != null); rowsNum = isTwoStepSubqry ? 0 : ins.rows().size(); @@ -164,10 +202,11 @@ else if (stmt instanceof GridSqlMerge) { desc = tbl.dataTable().rowDescriptor(); cols = merge.columns(); - sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query()); - if (sel == null) + if (noQuery(merge.rows())) elRows = merge.rows(); + else + sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query()); isTwoStepSubqry = (merge.query() != null); rowsNum = isTwoStepSubqry ? 0 : merge.rows().size(); @@ -233,7 +272,8 @@ else if (stmt instanceof GridSqlMerge) { String selectSql = sel != null ? sel.getSQL() : null; DmlDistributedPlanInfo distributed = (rowsNum == 0 && !F.isEmpty(selectSql)) ? - checkPlanCanBeDistributed(idx, conn, fieldsQuery, loc, selectSql, tbl.dataTable().cacheName()) : null; + checkPlanCanBeDistributed(idx, mvccEnabled, conn, fieldsQuery, loc, selectSql, + tbl.dataTable().cacheName()) : null; UpdateMode mode = stmt instanceof GridSqlMerge ? UpdateMode.MERGE : UpdateMode.INSERT; @@ -275,12 +315,38 @@ else if (stmt instanceof GridSqlMerge) { ); } + /** + * @param rows Insert rows. + * @return {@code True} if no query optimisation may be used. + */ + private static boolean noQuery(List rows) { + if (F.isEmpty(rows)) + return false; + + boolean noQry = true; + + for (int i = 0; i < rows.size(); i++) { + GridSqlElement[] row = rows.get(i); + + for (int i1 = 0; i1 < row.length; i1++) { + GridSqlElement el = row[i1]; + + if(!(noQry &= (el instanceof GridSqlConst || el instanceof GridSqlParameter))) + return noQry; + + } + } + + return true; + } + /** * Prepare update plan for UPDATE or DELETE. * * @param stmt UPDATE or DELETE statement. * @param loc Local query flag. * @param idx Indexing. + * @param mvccEnabled Mvcc flag. * @param conn Connection. * @param fieldsQuery Original query. * @param errKeysPos index to inject param for re-run keys at. Null if it's not a re-run plan. @@ -288,8 +354,8 @@ else if (stmt instanceof GridSqlMerge) { * @throws IgniteCheckedException if failed. */ private static UpdatePlan planForUpdate(GridSqlStatement stmt, boolean loc, IgniteH2Indexing idx, - @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQuery, @Nullable Integer errKeysPos) - throws IgniteCheckedException { + boolean mvccEnabled, @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQuery, + @Nullable Integer errKeysPos) throws IgniteCheckedException { GridSqlElement target; FastUpdate fastUpdate; @@ -319,9 +385,7 @@ else if (stmt instanceof GridSqlDelete) { GridH2Table h2Tbl = tbl.dataTable(); - if (h2Tbl == null) - throw new IgniteSQLException("Operation not supported for table '" + tbl.tableName() + "'", - IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + assert h2Tbl != null; GridH2RowDescriptor desc = h2Tbl.rowDescriptor(); @@ -381,7 +445,8 @@ else if (stmt instanceof GridSqlDelete) { String selectSql = sel.getSQL(); DmlDistributedPlanInfo distributed = F.isEmpty(selectSql) ? null : - checkPlanCanBeDistributed(idx, conn, fieldsQuery, loc, selectSql, tbl.dataTable().cacheName()); + checkPlanCanBeDistributed(idx, mvccEnabled, conn, fieldsQuery, loc, selectSql, + tbl.dataTable().cacheName()); return new UpdatePlan( UpdateMode.UPDATE, @@ -406,7 +471,8 @@ else if (stmt instanceof GridSqlDelete) { String selectSql = sel.getSQL(); DmlDistributedPlanInfo distributed = F.isEmpty(selectSql) ? null : - checkPlanCanBeDistributed(idx, conn, fieldsQuery, loc, selectSql, tbl.dataTable().cacheName()); + checkPlanCanBeDistributed(idx, mvccEnabled, conn, fieldsQuery, loc, selectSql, + tbl.dataTable().cacheName()); return new UpdatePlan( UpdateMode.DELETE, @@ -513,6 +579,7 @@ public static UpdatePlan planForBulkLoad(SqlBulkLoadCommand cmd, GridH2Table tbl * @param colIdx Column index if key or value is present in columns list, {@code -1} if it's not. * @param hasProps Whether column list affects individual properties of key or value. * @param key Whether supplier should be created for key or for value. + * @param forUpdate {@code FOR UPDATE} flag. * @return Closure returning key or value. * @throws IgniteCheckedException If failed. */ @@ -705,6 +772,7 @@ private static boolean updateAffectsKeyColumns(GridH2Table gridTbl, Set * Checks whether the given update plan can be distributed and returns additional info. * * @param idx Indexing. + * @param mvccEnabled Mvcc flag. * @param conn Connection. * @param fieldsQry Initial update query. * @param loc Local query flag. @@ -713,11 +781,10 @@ private static boolean updateAffectsKeyColumns(GridH2Table gridTbl, Set * @return distributed update plan info, or {@code null} if cannot be distributed. * @throws IgniteCheckedException if failed. */ - private static DmlDistributedPlanInfo checkPlanCanBeDistributed(IgniteH2Indexing idx, + private static DmlDistributedPlanInfo checkPlanCanBeDistributed(IgniteH2Indexing idx, boolean mvccEnabled, Connection conn, SqlFieldsQuery fieldsQry, boolean loc, String selectQry, String cacheName) throws IgniteCheckedException { - - if (loc || !isSkipReducerOnUpdateQuery(fieldsQry) || DmlUtils.isBatched(fieldsQry)) + if (loc || (!mvccEnabled && !isSkipReducerOnUpdateQuery(fieldsQry)) || DmlUtils.isBatched(fieldsQry)) return null; assert conn != null; @@ -732,7 +799,8 @@ private static DmlDistributedPlanInfo checkPlanCanBeDistributed(IgniteH2Indexing fieldsQry.getArgs(), fieldsQry.isCollocated(), fieldsQry.isDistributedJoins(), - fieldsQry.isEnforceJoinOrder(), idx); + fieldsQry.isEnforceJoinOrder(), + idx); boolean distributed = qry.skipMergeTable() && qry.mapQueries().size() == 1 && !qry.mapQueries().get(0).hasSubQueries(); 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 77bd69a4aeb48..6fd11c1fd7a84 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 @@ -28,6 +28,7 @@ import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; import org.apache.ignite.internal.processors.query.h2.H2Cursor; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeRequest; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeResponse; @@ -46,7 +47,6 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.logger.NullLogger; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.h2.engine.Session; import org.h2.index.BaseIndex; import org.h2.index.Cursor; @@ -266,15 +266,6 @@ public final int getDistributedMultiplier(Session ses, TableFilter[] filters, in return (GridH2Table)super.getTable(); } - /** - * @return Filter for currently running query or {@code null} if none. - */ - protected static IndexingQueryFilter threadLocalFilter() { - GridH2QueryContext qctx = GridH2QueryContext.get(); - - return qctx != null ? qctx.filter() : null; - } - /** {@inheritDoc} */ @Override public long getDiskSpaceUsed() { return 0; @@ -421,7 +412,7 @@ private void onIndexRangeRequest(final ClusterNode node, final GridH2IndexRangeR // This is the first request containing all the search rows. assert !msg.bounds().isEmpty() : "empty bounds"; - src = new RangeSource(msg.bounds(), msg.segment(), qctx.filter()); + src = new RangeSource(msg.bounds(), msg.segment(), filter(qctx)); } else { // This is request to fetch next portion of data. @@ -474,6 +465,14 @@ else if (msg.bounds() == null) { send(singletonList(node), res); } + /** + * @param qctx Query context. + * @return Row filter. + */ + protected BPlusTree.TreeRowClosure filter(GridH2QueryContext qctx) { + throw new UnsupportedOperationException(); + } + /** * @param node Responded node. * @param msg Response message. @@ -1463,20 +1462,17 @@ private class RangeSource { private final int segment; /** */ - final IndexingQueryFilter filter; + private final BPlusTree.TreeRowClosure filter; /** Iterator. */ Iterator iter = emptyIterator(); /** * @param bounds Bounds. + * @param segment Segment. * @param filter Filter. */ - RangeSource( - Iterable bounds, - int segment, - IndexingQueryFilter filter - ) { + RangeSource(Iterable bounds, int segment, BPlusTree.TreeRowClosure filter) { this.segment = segment; this.filter = filter; boundsIter = bounds.iterator(); @@ -1536,7 +1532,7 @@ public GridH2RowRange next(int maxRows) { IgniteTree t = treeForRead(segment); - iter = new CursorIteratorWrapper(doFind0(t, first, true, last, filter)); + iter = new CursorIteratorWrapper(doFind0(t, first, last, filter)); if (!iter.hasNext()) { // We have to return empty range here. @@ -1561,7 +1557,6 @@ protected IgniteTree treeForRead(int segment) { /** * @param t Tree. * @param first Lower bound. - * @param includeFirst Whether lower bound should be inclusive. * @param last Upper bound always inclusive. * @param filter Filter. * @return Iterator over rows in given range. @@ -1569,9 +1564,8 @@ protected IgniteTree treeForRead(int segment) { protected H2Cursor doFind0( IgniteTree t, @Nullable SearchRow first, - boolean includeFirst, @Nullable SearchRow last, - IndexingQueryFilter filter) { + BPlusTree.TreeRowClosure filter) { throw new UnsupportedOperationException(); } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java index 291f8c84af80c..e15205438f058 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.query.h2.opt; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.h2.value.Value; @@ -60,4 +61,14 @@ public GridH2KeyRowOnheap(CacheDataRow row, Value key) { @Override public long expireTime() { return 0; } + + /** {@inheritDoc} */ + @Override public int size() throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public int headerSize() { + throw new UnsupportedOperationException(); + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java index e8555367908d4..f966034cb0774 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java @@ -65,8 +65,10 @@ public class GridH2KeyValueRowOnheap extends GridH2Row { * @param valType Value type. * @throws IgniteCheckedException If failed. */ - public GridH2KeyValueRowOnheap(GridH2RowDescriptor desc, CacheDataRow row, int keyType, int valType) - throws IgniteCheckedException { + public GridH2KeyValueRowOnheap(GridH2RowDescriptor desc, + CacheDataRow row, + int keyType, + int valType) throws IgniteCheckedException { super(row); this.desc = desc; @@ -80,6 +82,11 @@ public GridH2KeyValueRowOnheap(GridH2RowDescriptor desc, CacheDataRow row, int k this.ver = desc.wrap(row.version(), Value.JAVA_OBJECT); } + /** {@inheritDoc} */ + @Override public boolean indexSearchRow() { + return false; + } + /** {@inheritDoc} */ @Override public int getColumnCount() { return DEFAULT_COLUMNS_COUNT + desc.fieldsCount(); @@ -220,4 +227,14 @@ private void setCached(int colIdx, Value val) { @Override public final int hashCode() { throw new UnsupportedOperationException(); } + + /** {@inheritDoc} */ + @Override public int size() throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public int headerSize() { + throw new UnsupportedOperationException(); + } } \ No newline at end of file diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java index 00312b80b4bd7..b008c081f878d 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java @@ -284,6 +284,11 @@ private static class MetaRow extends GridH2SearchRowAdapter { throw new IllegalStateException("Index: " + idx); } } + + /** {@inheritDoc} */ + @Override public boolean indexSearchRow() { + return false; + } } /** diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java index fd8a613247baf..d24dc0881815b 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PlainRowFactory.java @@ -70,7 +70,7 @@ private static final class RowKey extends GridH2SearchRowAdapter { /** * @param key Key. */ - public RowKey(Value key) { + RowKey(Value key) { this.key = key; } @@ -91,6 +91,11 @@ public RowKey(Value key) { key = v; } + /** {@inheritDoc} */ + @Override public boolean indexSearchRow() { + return true; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(RowKey.class, this); @@ -137,6 +142,11 @@ private RowPair(Value v1, Value v2) { } } + /** {@inheritDoc} */ + @Override public boolean indexSearchRow() { + return true; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(RowPair.class, this); @@ -173,6 +183,11 @@ private RowSimple(Value[] vals) { vals[idx] = v; } + /** {@inheritDoc} */ + @Override public boolean indexSearchRow() { + return true; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(RowSimple.class, this); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java index 7b52ea46ee975..f12c0f30170fd 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -84,6 +85,9 @@ public class GridH2QueryContext { /** */ private GridH2CollocationModel qryCollocationMdl; + /** */ + private MvccSnapshot mvccSnapshot; + /** */ private MapQueryLazyWorker lazyWorker; @@ -106,12 +110,33 @@ public GridH2QueryContext(UUID locNodeId, UUID nodeId, long qryId, GridH2QueryTy * @param segmentId Index segment ID. * @param type Query type. */ - public GridH2QueryContext(UUID locNodeId, UUID nodeId, long qryId, int segmentId, GridH2QueryType type) { + public GridH2QueryContext(UUID locNodeId, + UUID nodeId, + long qryId, + int segmentId, + GridH2QueryType type) { assert segmentId == 0 || type == MAP; key = new Key(locNodeId, nodeId, qryId, segmentId, type); } + /** + * @return Mvcc snapshot. + */ + @Nullable public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + + /** + * @param mvccSnapshot Mvcc snapshot. + * @return {@code this}. + */ + public GridH2QueryContext mvccSnapshot(MvccSnapshot mvccSnapshot) { + this.mvccSnapshot = mvccSnapshot; + + return this; + } + /** * @return Type. */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java index 8b1b7115327b5..2133e1a86e5bd 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java @@ -27,7 +27,7 @@ */ public abstract class GridH2Row extends GridH2SearchRowAdapter implements CacheDataRow { /** Row. */ - private CacheDataRow row; + protected final CacheDataRow row; /** * @param row Row. @@ -78,11 +78,57 @@ public abstract class GridH2Row extends GridH2SearchRowAdapter implements CacheD /** {@inheritDoc} */ @Override public int hash() { - throw new UnsupportedOperationException(); + return row.hash(); } /** {@inheritDoc} */ @Override public int cacheId() { return row.cacheId(); } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return row.mvccCoordinatorVersion(); + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return row.mvccCounter(); + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return row.mvccOperationCounter(); + } + + /** {@inheritDoc} */ + public byte mvccTxState() { + return row.mvccTxState(); + } + + /** {@inheritDoc} */ + @Override public long newMvccCoordinatorVersion() { + return row.newMvccCoordinatorVersion(); + } + + /** {@inheritDoc} */ + @Override public long newMvccCounter() { + return row.newMvccCounter(); + } + + /** {@inheritDoc} */ + @Override public int newMvccOperationCounter() { + return row.newMvccOperationCounter(); + } + + /** {@inheritDoc} */ + @Override public byte newMvccTxState() { + return row.newMvccTxState(); + } + + + /** {@inheritDoc} */ + @Override public boolean indexSearchRow() { + return false; + } } \ No newline at end of file diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java index 1d915e583f450..23f3ba4f26502 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java @@ -29,6 +29,8 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.GridQueryProperty; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; @@ -36,7 +38,6 @@ import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.h2.message.DbException; import org.h2.result.SearchRow; -import org.h2.result.SimpleRow; import org.h2.util.LocalDateTimeUtils; import org.h2.value.DataType; import org.h2.value.Value; @@ -58,6 +59,7 @@ import org.h2.value.ValueTime; import org.h2.value.ValueTimestamp; import org.h2.value.ValueUuid; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL; @@ -280,8 +282,9 @@ public GridH2Row createRow(CacheDataRow dataRow) throws IgniteCheckedException { GridH2Row row; try { - if (dataRow.value() == null) // Only can happen for remove operation, can create simple search row. + if (dataRow.value() == null) { // Only can happen for remove operation, can create simple search row. row = new GridH2KeyRowOnheap(dataRow, wrap(dataRow.key(), keyType)); + } else row = new GridH2KeyValueRowOnheap(this, dataRow, keyType, valType); } @@ -465,7 +468,7 @@ public SearchRow prepareProxyIndexRow(SearchRow row) { copyAliasColumnData(data, KEY_COL, keyAliasColId); copyAliasColumnData(data, VAL_COL, valAliasColId); - return new SimpleRow(data); + return GridH2PlainRowFactory.create(data); } /** diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRow.java new file mode 100644 index 0000000000000..5de6216109553 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRow.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2.opt; + +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; +import org.apache.ignite.internal.processors.query.h2.database.H2Tree; +import org.h2.result.Row; + +/** + * + */ +public interface GridH2SearchRow extends Row, MvccVersionAware { + /** + * @return {@code True} for rows used for index search (as opposed to rows stored in {@link H2Tree}. + */ + public boolean indexSearchRow(); +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java index 24a90b3115131..2e512b1d820ec 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SearchRowAdapter.java @@ -17,15 +17,20 @@ package org.apache.ignite.internal.processors.query.h2.opt; +import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.h2.result.Row; import org.h2.result.SearchRow; import org.h2.store.Data; import org.h2.value.Value; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA; + /** * Dummy H2 search row adadpter. */ -public abstract class GridH2SearchRowAdapter implements Row { +public abstract class GridH2SearchRowAdapter implements GridH2SearchRow { /** {@inheritDoc} */ @Override public void setKeyAndVersion(SearchRow old) { throw new UnsupportedOperationException(); @@ -100,4 +105,24 @@ public abstract class GridH2SearchRowAdapter implements Row { @Override public Value[] getValueList() { throw new UnsupportedOperationException(); } + + /** {@inheritDoc} */ + @Override public long mvccCoordinatorVersion() { + return MVCC_CRD_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public long mvccCounter() { + return MVCC_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public int mvccOperationCounter() { + return MVCC_OP_COUNTER_NA; + } + + /** {@inheritDoc} */ + @Override public byte mvccTxState() { + return TxState.NA; + } } 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 15be253132724..a612b637980ad 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 @@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; @@ -442,12 +441,12 @@ private GridH2IndexBase pk() { * @param prevRowAvailable Whether previous row is available. * @throws IgniteCheckedException If failed. */ - public void update(CacheDataRow row, @Nullable CacheDataRow prevRow, boolean prevRowAvailable) - throws IgniteCheckedException { + public void update(CacheDataRow row, @Nullable CacheDataRow prevRow, boolean prevRowAvailable) throws IgniteCheckedException { assert desc != null; GridH2KeyValueRowOnheap row0 = (GridH2KeyValueRowOnheap)desc.createRow(row); - GridH2KeyValueRowOnheap prevRow0 = prevRow != null ? (GridH2KeyValueRowOnheap)desc.createRow(prevRow) : null; + GridH2KeyValueRowOnheap prevRow0 = prevRow != null ? (GridH2KeyValueRowOnheap)desc.createRow(prevRow) : + null; row0.prepareValuesCache(); @@ -539,7 +538,6 @@ public boolean remove(CacheDataRow row) throws IgniteCheckedException { /** * Add row to index. - * * @param idx Index to add row to. * @param row Row to add to index. * @param prevRow Previous row state, if any. 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 133333e265c27..599d7920ff2e0 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 @@ -19,6 +19,7 @@ import java.lang.reflect.Field; import java.sql.PreparedStatement; +import java.sql.SQLException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -42,6 +43,7 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgniteUuid; import org.h2.command.Command; import org.h2.command.CommandContainer; import org.h2.command.CommandInterface; @@ -99,6 +101,7 @@ import org.h2.table.TableFilter; import org.h2.table.TableView; import org.h2.value.DataType; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperationType.AND; @@ -151,6 +154,15 @@ public class GridSqlQueryParser { /** */ private static final Getter GROUP_INDEXES = getter(Select.class, "groupIndex"); + /** */ + private static final Getter SELECT_IS_FOR_UPDATE = getter(Select.class, "isForUpdate"); + + /** */ + private static final Getter SELECT_IS_GROUP_QUERY = getter(Select.class, "isGroupQuery"); + + /** */ + private static final Getter UNION_IS_FOR_UPDATE = getter(SelectUnion.class, "isForUpdate"); + /** */ private static final Getter OPERATION_TYPE = getter(Operation.class, "opType"); @@ -527,7 +539,7 @@ public GridSqlQueryParser(boolean useOptimizedSubqry) { * @return {@code true} in case of multiple statements. */ public static boolean checkMultipleStatements(PreparedStatement stmt) { - Command cmd = COMMAND.get((JdbcPreparedStatement)stmt); + Command cmd = extractCommand(stmt); return ORG_H2_COMMAND_COMMAND_LIST.equals(cmd.getClass().getName()); } @@ -537,7 +549,7 @@ public static boolean checkMultipleStatements(PreparedStatement stmt) { * @return Parsed select. */ public static Prepared prepared(PreparedStatement stmt) { - Command cmd = COMMAND.get((JdbcPreparedStatement)stmt); + Command cmd = extractCommand(stmt); assert cmd instanceof CommandContainer; @@ -549,21 +561,99 @@ public static Prepared prepared(PreparedStatement stmt) { * @return Parsed select. */ public static PreparedWithRemaining preparedWithRemaining(PreparedStatement stmt) { - Command cmd = COMMAND.get((JdbcPreparedStatement)stmt); + Command cmd = extractCommand(stmt); if (cmd instanceof CommandContainer) return new PreparedWithRemaining(PREPARED.get(cmd), null); else { Class cmdCls = cmd.getClass(); - if (cmdCls.getName().equals(ORG_H2_COMMAND_COMMAND_LIST)) { + if (cmdCls.getName().equals(ORG_H2_COMMAND_COMMAND_LIST)) return new PreparedWithRemaining(PREPARED.get(LIST_COMMAND.get(cmd)), REMAINING.get(cmd)); - } else throw new IgniteSQLException("Unexpected statement command"); } } + /** */ + private static Command extractCommand(PreparedStatement stmt) { + try { + return COMMAND.get(stmt.unwrap(JdbcPreparedStatement.class)); + } catch (SQLException e) { + throw new IgniteSQLException(e); + } + } + + /** + * @param p Prepared. + * @return Whether {@code p} is an {@code SELECT FOR UPDATE} query. + */ + public static boolean isForUpdateQuery(Prepared p) { + boolean union; + + if (p.getClass() == Select.class) + union = false; + else if (p.getClass() == SelectUnion.class) + union = true; + else + return false; + + boolean forUpdate = (!union && SELECT_IS_FOR_UPDATE.get((Select)p)) || + (union && UNION_IS_FOR_UPDATE.get((SelectUnion)p)); + + if (union && forUpdate) + throw new IgniteSQLException("SELECT UNION FOR UPDATE is not supported.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + return forUpdate; + } + + /** + * @param p Statement to rewrite, if needed. + * @param inTx Whether there is an active transaction. + * @return Query with {@code key} and {@code val} columns appended to the list of columns, + * if it's an {@code FOR UPDATE} query, or {@code null} if nothing has to be done. + */ + @NotNull public static String rewriteQueryForUpdateIfNeeded(Prepared p, boolean inTx) { + GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(p); + return rewriteQueryForUpdateIfNeeded(gridStmt, inTx); + } + + /** + * @param stmt Statement to rewrite, if needed. + * @param inTx Whether there is an active transaction. + * @return Query with {@code key} and {@code val} columns appended to the list of columns, + * if it's an {@code FOR UPDATE} query, or {@code null} if nothing has to be done. + */ + @NotNull public static String rewriteQueryForUpdateIfNeeded(GridSqlStatement stmt, boolean inTx) { + // We have checked above that it's not an UNION query, so it's got to be SELECT. + assert stmt instanceof GridSqlSelect; + + GridSqlSelect sel = (GridSqlSelect)stmt; + + // How'd we get here otherwise? + assert sel.isForUpdate(); + + if (inTx) { + GridSqlAst from = sel.from(); + + GridSqlTable gridTbl = from instanceof GridSqlTable ? (GridSqlTable)from : + ((GridSqlAlias)from).child(); + + GridH2Table tbl = gridTbl.dataTable(); + + Column keyCol = tbl.getColumn(0); + + sel.addColumn(new GridSqlAlias("_key_" + IgniteUuid.vmId(), + new GridSqlColumn(keyCol, null, keyCol.getName()), true), true); + } + + // We need to remove this flag for final flag we'll feed to H2. + sel.forUpdate(false); + + return sel.getSQL(); + } + /** * @param qry Query expression to parse. * @return Subquery AST. @@ -611,7 +701,7 @@ private GridSqlElement parseTable(Table tbl) { // We can't cache simple tables because otherwise it will be the same instance for all // table filters. Thus we will not be able to distinguish one table filter from another. // Table here is semantically equivalent to a table filter. - if (tbl instanceof TableBase) + if (tbl instanceof TableBase || tbl instanceof MetaTable) return new GridSqlTable(tbl); // Other stuff can be cached because we will have separate instances in @@ -662,6 +752,8 @@ private GridSqlSelect parseSelect(Select select) { TableFilter filter = select.getTopTableFilter(); + boolean isForUpdate = SELECT_IS_FOR_UPDATE.get(select); + do { assert0(filter != null, select); assert0(filter.getNestedJoin() == null, select); @@ -693,6 +785,30 @@ private GridSqlSelect parseSelect(Select select) { res.from(from); + if (isForUpdate) { + if (!(from instanceof GridSqlTable || + (from instanceof GridSqlAlias && from.size() == 1 && from.child() instanceof GridSqlTable))) + throw new IgniteSQLException("SELECT FOR UPDATE with joins is not supported.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + GridSqlTable gridTbl = from instanceof GridSqlTable ? (GridSqlTable)from : + ((GridSqlAlias)from).child(); + + GridH2Table tbl = gridTbl.dataTable(); + + if (tbl == null) + throw new IgniteSQLException("SELECT FOR UPDATE query must involve Ignite table.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + if (select.getLimit() != null || select.getOffset() != null) + throw new IgniteSQLException("LIMIT/OFFSET clauses are not supported for SELECT FOR UPDATE.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + + if (SELECT_IS_GROUP_QUERY.get(select)) + throw new IgniteSQLException("SELECT FOR UPDATE with aggregates and/or GROUP BY is not supported.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + } + ArrayList expressions = select.getExpressions(); for (int i = 0; i < expressions.size(); i++) @@ -708,6 +824,8 @@ private GridSqlSelect parseSelect(Select select) { if (havingIdx >= 0) res.havingColumn(havingIdx); + res.forUpdate(isForUpdate); + processSortOrder(select.getSortOrder(), res); res.limit(parseExpression(select.getLimit(), false)); @@ -1557,6 +1675,29 @@ public static Query query(Prepared qry) { throw new CacheException("Unsupported query: " + qry); } + /** + * @param stmt Prepared. + * @return Target table. + */ + @NotNull public static GridH2Table dmlTable(@NotNull Prepared stmt) { + Table table; + + if (stmt.getClass() == Insert.class) + table = INSERT_TABLE.get((Insert)stmt); + else if (stmt.getClass() == Merge.class) + table = MERGE_TABLE.get((Merge)stmt); + else if (stmt.getClass() == Delete.class) + table = DELETE_FROM.get((Delete)stmt).getTable(); + else if (stmt.getClass() == Update.class) + table = UPDATE_TARGET.get((Update)stmt).getTable(); + else + throw new IgniteException("Unsupported statement: " + stmt); + + assert table instanceof GridH2Table : table; + + return (GridH2Table) table; + } + /** * Check if query may be run locally on all caches mentioned in the query. * @param replicatedOnlyQry replicated-only query flag from original {@link SqlFieldsQuery}. @@ -1657,6 +1798,13 @@ public final GridSqlStatement parse(Prepared stmt) { throw new CacheException("Unsupported SQL statement: " + stmt); } + /** + * @return H2 to Grid objects map. + */ + public Map objectsMap() { + return h2ObjToGridObj; + } + /** * @param qry Query. * @return Parsed query AST. @@ -1676,6 +1824,10 @@ private GridSqlQuery parseQuery(Query qry) { * @return Parsed AST. */ private GridSqlUnion parseUnion(SelectUnion union) { + if (UNION_IS_FOR_UPDATE.get(union)) + throw new IgniteSQLException("SELECT UNION FOR UPDATE is not supported.", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION); + GridSqlUnion res = (GridSqlUnion)h2ObjToGridObj.get(union); if (res != null) 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 5c320055bcdf8..3e3b449ecc756 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 @@ -220,8 +220,10 @@ public static GridCacheTwoStepQuery split( qry = parse(optimize(h2, conn, qry.getSQL(), params, false, enforceJoinOrder), true); + boolean forUpdate = GridSqlQueryParser.isForUpdateQuery(prepared); + // Do the actual query split. We will update the original query AST, need to be careful. - splitter.splitQuery(qry); + splitter.splitQuery(qry, forUpdate); assert !F.isEmpty(splitter.mapSqlQrys): "map"; // We must have at least one map query. assert splitter.rdcSqlQry != null: "rdc"; // We must have a reduce query. @@ -261,13 +263,16 @@ public static GridCacheTwoStepQuery split( // all map queries must have non-empty derivedPartitions to use this feature. twoStepQry.derivedPartitions(mergePartitionsFromMultipleQueries(twoStepQry.mapQueries())); + twoStepQry.forUpdate(forUpdate); + return twoStepQry; } /** * @param qry Optimized and normalized query to split. + * @param forUpdate {@code SELECT FOR UPDATE} flag. */ - private void splitQuery(GridSqlQuery qry) throws IgniteCheckedException { + private void splitQuery(GridSqlQuery qry, boolean forUpdate) throws IgniteCheckedException { // Create a fake parent AST element for the query to allow replacing the query in the parent by split. GridSqlSubquery fakeQryPrnt = new GridSqlSubquery(qry); @@ -308,6 +313,15 @@ else if (!qrym.needSplit) // Just split the top level query. // Get back the updated query from the fake parent. It will be our reduce query. qry = fakeQryPrnt.subquery(); + // Reset SELECT FOR UPDATE flag for reduce query. + if (forUpdate) { + assert qry instanceof GridSqlSelect; + + GridSqlSelect sel = (GridSqlSelect)qry; + + sel.forUpdate(false); + } + String rdcQry = qry.getSQL(); checkNoDataTablesInReduceQuery(qry, rdcQry); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java index bfa008926aa41..8ea61e156e33c 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java @@ -56,6 +56,9 @@ public class GridSqlSelect extends GridSqlQuery { /** */ private int havingCol = -1; + /** */ + private boolean isForUpdate; + /** * @param colIdx Column index as for {@link #column(int)}. * @return Child index for {@link #child(int)}. @@ -167,6 +170,9 @@ public int allColumns() { getSortLimitSQL(buff); + if (isForUpdate) + buff.append("\nFOR UPDATE"); + return buff.toString(); } @@ -363,6 +369,20 @@ public GridSqlSelect havingColumn(int col) { return this; } + /** + * @return Whether this statement is {@code FOR UPDATE}. + */ + public boolean isForUpdate() { + return isForUpdate; + } + + /** + * @param forUpdate Whether this statement is {@code FOR UPDATE}. + */ + public void forUpdate(boolean forUpdate) { + isForUpdate = forUpdate; + } + /** * @return Index of HAVING column. */ 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 d348e21126208..39c12c65792da 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 @@ -18,7 +18,9 @@ package org.apache.ignite.internal.processors.query.h2.twostep; import java.sql.Connection; +import java.sql.PreparedStatement; import java.sql.ResultSet; +import java.sql.SQLException; import java.util.AbstractCollection; import java.util.ArrayList; import java.util.Arrays; @@ -33,6 +35,7 @@ import java.util.concurrent.ConcurrentMap; 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; @@ -47,25 +50,33 @@ import org.apache.ignite.events.EventType; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.GridTopic; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.CompoundLockFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsReservation; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; import org.apache.ignite.internal.processors.query.GridQueryCancel; +import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.h2.H2Utils; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.processors.query.h2.ResultSetEnlistFuture; import org.apache.ignite.internal.processors.query.h2.UpdateResult; import org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode; import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext; import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; +import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest; @@ -73,14 +84,19 @@ import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2DmlRequest; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2DmlResponse; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2SelectForUpdateTxDetails; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.apache.ignite.thread.IgniteThread; +import org.h2.command.Prepared; import org.h2.jdbc.JdbcResultSet; import org.h2.value.Value; import org.jetbrains.annotations.Nullable; @@ -482,26 +498,6 @@ private static Collection wrap(final int[] ints) { }; } - /** - * @param caches Cache IDs. - * @return The first found partitioned cache. - */ - private GridCacheContext findFirstPartitioned(List caches) { - GridCacheSharedContext sctx = ctx.cache().context(); - - for (int i = 0; i < caches.size(); i++) { - GridCacheContext mainCctx = sctx.cacheContext(caches.get(i)); - - if (mainCctx == null) - throw new CacheException("Failed to find cache."); - - if (!mainCctx.isLocal() && !mainCctx.isReplicated()) - return mainCctx; - } - - throw new IllegalStateException("Failed to find a partitioned cache."); - } - /** * @param node Node. * @param req Query request. @@ -525,10 +521,73 @@ private void onQueryRequest(final ClusterNode node, final GridH2QueryRequest req final List cacheIds = req.caches(); int segments = explain || replicated || F.isEmpty(cacheIds) ? 1 : - findFirstPartitioned(cacheIds).config().getQueryParallelism(); + CU.firstPartitioned(ctx.cache().context(), cacheIds).config().getQueryParallelism(); final Object[] params = req.parameters(); + final GridDhtTxLocalAdapter tx; + + GridH2SelectForUpdateTxDetails txReq = req.txDetails(); + + try { + if (txReq != null) { + // Prepare to run queries. + GridCacheContext mainCctx = mainCacheContext(cacheIds); + + if (mainCctx == null || mainCctx.atomic() || !mainCctx.mvccEnabled() || cacheIds.size() != 1) + throw new IgniteSQLException("SELECT FOR UPDATE is supported only for queries " + + "that involve single transactional cache."); + + GridDhtTransactionalCacheAdapter txCache = (GridDhtTransactionalCacheAdapter)mainCctx.cache(); + + if (!node.isLocal()) { + tx = txCache.initTxTopologyVersion( + node.id(), + node, + txReq.version(), + txReq.futureId(), + txReq.miniId(), + txReq.firstClientRequest(), + req.topologyVersion(), + txReq.threadId(), + txReq.timeout(), + txReq.subjectId(), + txReq.taskNameHash()); + } + else { + tx = MvccUtils.tx(ctx, txReq.version()); + + assert tx != null; + } + } + else + tx = null; + } + catch (IgniteException | IgniteCheckedException e) { + // send error if TX was not initialized properly. + releaseReservations(); + + U.error(log, "Failed to execute local query.", e); + + sendError(node, req.requestId(), e); + + return; + } + + AtomicInteger runCntr; + CompoundLockFuture lockFut; + + if (txReq != null && segments > 1) { + runCntr = new AtomicInteger(segments); + lockFut = new CompoundLockFuture(segments, tx); + + lockFut.init(); + } + else { + runCntr = null; + lockFut = null; + } + for (int i = 1; i < segments; i++) { assert !F.isEmpty(cacheIds); @@ -550,13 +609,17 @@ private void onQueryRequest(final ClusterNode node, final GridH2QueryRequest req false, // Replicated is always false here (see condition above). req.timeout(), params, - true); // Lazy = true. + true, + req.mvccSnapshot(), + tx, + txReq, + lockFut, + runCntr); } else { ctx.closure().callLocal( new Callable() { - @Override - public Void call() throws Exception { + @Override public Void call() { onQueryRequest0(node, req.requestId(), segment, @@ -572,7 +635,12 @@ public Void call() throws Exception { false, req.timeout(), params, - false); // Lazy = false. + false, + req.mvccSnapshot(), + tx, + txReq, + lockFut, + runCntr); return null; } @@ -596,7 +664,11 @@ public Void call() throws Exception { replicated, req.timeout(), params, - lazy); + lazy, + req.mvccSnapshot(), + tx, + txReq, + lockFut, runCntr); } /** @@ -612,6 +684,11 @@ public Void call() throws Exception { * @param pageSize Page size. * @param distributedJoinMode Query distributed join mode. * @param lazy Streaming flag. + * @param mvccSnapshot MVCC snapshot. + * @param tx Transaction. + * @param txDetails TX details, if it's a {@code FOR UPDATE} request, or {@code null}. + * @param lockFut Lock future. + * @param runCntr Counter which counts remaining queries in case segmented index is used. */ private void onQueryRequest0( final ClusterNode node, @@ -629,10 +706,19 @@ private void onQueryRequest0( final boolean replicated, final int timeout, final Object[] params, - boolean lazy - ) { + boolean lazy, + @Nullable final MvccSnapshot mvccSnapshot, + @Nullable final GridDhtTxLocalAdapter tx, + @Nullable final GridH2SelectForUpdateTxDetails txDetails, + @Nullable final CompoundLockFuture lockFut, + @Nullable final AtomicInteger runCntr) { MapQueryLazyWorker worker = MapQueryLazyWorker.currentWorker(); + // In presence of TX, we also must always have matching details. + assert tx == null || txDetails != null; + + boolean inTx = (tx != null); + if (lazy && worker == null) { // Lazy queries must be re-submitted to dedicated workers. MapQueryLazyWorkerKey key = new MapQueryLazyWorkerKey(node.id(), reqId, segmentId); @@ -640,8 +726,28 @@ private void onQueryRequest0( worker.submit(new Runnable() { @Override public void run() { - onQueryRequest0(node, reqId, segmentId, schemaName, qrys, cacheIds, topVer, partsMap, parts, - pageSize, distributedJoinMode, enforceJoinOrder, replicated, timeout, params, true); + onQueryRequest0( + node, + reqId, + segmentId, + schemaName, + qrys, + cacheIds, + topVer, + partsMap, + parts, + pageSize, + distributedJoinMode, + enforceJoinOrder, + replicated, + timeout, + params, + true, + mvccSnapshot, + tx, + txDetails, + lockFut, + runCntr); } }); @@ -666,9 +772,11 @@ private void onQueryRequest0( return; } + if (lazy && txDetails != null) + throw new IgniteSQLException("Lazy execution of SELECT FOR UPDATE queries is not supported."); + // Prepare to run queries. - GridCacheContext mainCctx = - !F.isEmpty(cacheIds) ? ctx.cache().context().cacheContext(cacheIds.get(0)) : null; + GridCacheContext mainCctx = mainCacheContext(cacheIds); MapNodeResults nodeRess = resultsForNode(node.id()); @@ -677,7 +785,9 @@ private void onQueryRequest0( List reserved = new ArrayList<>(); try { - if (topVer != null) { + // We want to reserve only in not SELECT FOR UPDATE case - + // otherwise, their state is protected by locked topology. + if (topVer != null && txDetails == null) { // Reserve primary for topology version or explicit partitions. String err = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); @@ -692,7 +802,7 @@ private void onQueryRequest0( } } - qr = new MapQueryResults(h2, reqId, qrys.size(), mainCctx, MapQueryLazyWorker.currentWorker()); + qr = new MapQueryResults(h2, reqId, qrys.size(), mainCctx, MapQueryLazyWorker.currentWorker(), inTx); if (nodeRess.put(reqId, segmentId, qr) != null) throw new IllegalStateException(); @@ -709,6 +819,7 @@ private void onQueryRequest0( .pageSize(pageSize) .topologyVersion(topVer) .reservations(reserved) + .mvccSnapshot(mvccSnapshot) .lazyWorker(worker); Connection conn = h2.connectionForSchema(schemaName); @@ -744,12 +855,56 @@ private void onQueryRequest0( for (GridCacheSqlQuery qry : qrys) { ResultSet rs = null; + boolean removeMapping = false; + // If we are not the target node for this replicated query, just ignore it. if (qry.node() == null || (segmentId == 0 && qry.node().equals(ctx.localNodeId()))) { - rs = h2.executeSqlQueryWithTimer(conn, qry.query(), - F.asList(qry.parameters(params)), true, - timeout, - qr.queryCancel(qryIdx)); + String sql = qry.query(); Collection params0 = F.asList(qry.parameters(params)); + + PreparedStatement stmt; + + try { + stmt = h2.prepareStatement(conn, sql, true); + } + catch (SQLException e) { + throw new IgniteCheckedException("Failed to parse SQL query: " + sql, e); + } + + Prepared p = GridSqlQueryParser.prepared(stmt); + + if (GridSqlQueryParser.isForUpdateQuery(p)) { + sql = GridSqlQueryParser.rewriteQueryForUpdateIfNeeded(p, inTx); + stmt = h2.prepareStatement(conn, sql, true); + } + + h2.bindParameters(stmt, params0); + + rs = h2.executeSqlQueryWithTimer(stmt, conn, sql, params0, timeout, qr.queryCancel(qryIdx)); + + if (inTx) { + ResultSetEnlistFuture enlistFut = ResultSetEnlistFuture.future( + ctx.localNodeId(), + txDetails.version(), + mvccSnapshot, + txDetails.threadId(), + IgniteUuid.randomUuid(), + txDetails.miniId(), + parts, + tx, + timeout, + mainCctx, + rs + ); + + if (lockFut != null) + lockFut.register(enlistFut); + + enlistFut.init(); + + enlistFut.get(); + + rs.beforeFirst(); + } if (evt) { ctx.event().record(new CacheQueryExecutedEvent<>( @@ -778,8 +933,35 @@ private void onQueryRequest0( throw new QueryCancelledException(); } + if (inTx) { + if (tx.dht() && (runCntr == null || runCntr.decrementAndGet() == 0)) { + if (removeMapping = tx.empty() && !tx.queryEnlisted()) + tx.rollbackAsync().get(); + } + } + // Send the first page. - sendNextPage(nodeRess, node, qr, qryIdx, segmentId, pageSize); + if (lockFut == null) + sendNextPage(nodeRess, node, qr, qryIdx, segmentId, pageSize, removeMapping); + else { + GridQueryNextPageResponse msg = prepareNextPage(nodeRess, node, qr, qryIdx, segmentId, pageSize, removeMapping); + + if (msg != null) { + lockFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture future) { + try { + if (node.isLocal()) + h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg); + else + ctx.io().sendToGridTopic(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL); + } + catch (Exception e) { + U.error(log, e); + } + } + }); + } + } qryIdx++; } @@ -835,6 +1017,14 @@ private void onQueryRequest0( } } + /** + * @param cacheIds Cache ids. + * @return Id of the first cache in list, or {@code null} if list is empty. + */ + private GridCacheContext mainCacheContext(List cacheIds) { + return !F.isEmpty(cacheIds) ? ctx.cache().context().cacheContext(cacheIds.get(0)) : null; + } + /** * Releases reserved partitions. */ @@ -899,7 +1089,7 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th final boolean replicated = req.isFlagSet(GridH2QueryRequest.FLAG_REPLICATED); if (!replicated && !F.isEmpty(cacheIds) && - findFirstPartitioned(cacheIds).config().getQueryParallelism() > 1) { + CU.firstPartitioned(ctx.cache().context(), cacheIds).config().getQueryParallelism() > 1) { fldsQry.setDistributedJoins(true); local = false; @@ -1031,12 +1221,12 @@ else if (qr.cancelled()) if (lazyWorker != null) { lazyWorker.submit(new Runnable() { @Override public void run() { - sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize()); + sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize(), false); } }); } else - sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize()); + sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize(), false); } } @@ -1047,15 +1237,18 @@ else if (qr.cancelled()) * @param qry Query. * @param segmentId Index segment ID. * @param pageSize Page size. + * @param removeMapping Remove mapping flag. + * @return Next page. + * @throws IgniteCheckedException If failed. */ - private void sendNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryResults qr, int qry, int segmentId, - int pageSize) { + private GridQueryNextPageResponse prepareNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryResults qr, int qry, int segmentId, + int pageSize, boolean removeMapping) throws IgniteCheckedException { MapQueryResult res = qr.result(qry); assert res != null; if (res.closed()) - return; + return null; int page = res.page(); @@ -1075,20 +1268,45 @@ private void sendNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryRes } } - try { - boolean loc = node.isLocal(); + boolean loc = node.isLocal(); - GridQueryNextPageResponse msg = new GridQueryNextPageResponse(qr.queryRequestId(), segmentId, qry, page, - page == 0 ? res.rowCount() : -1, - res.columnCount(), - loc ? null : toMessages(rows, new ArrayList(res.columnCount())), - loc ? rows : null, - last); + // In case of SELECT FOR UPDATE the last columns is _KEY, + // we can't retrieve them for an arbitrary row otherwise. + int colsCnt = !qr.isForUpdate() ? res.columnCount() : res.columnCount() - 1; - if (loc) - h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg); - else - ctx.io().sendToGridTopic(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL); + GridQueryNextPageResponse msg = new GridQueryNextPageResponse(qr.queryRequestId(), segmentId, qry, page, + page == 0 ? res.rowCount() : -1, + colsCnt, + loc ? null : toMessages(rows, new ArrayList<>(res.columnCount()), colsCnt), + loc ? rows : null, + last); + + msg.removeMapping(removeMapping); + + return msg; + } + + /** + * @param nodeRess Results. + * @param node Node. + * @param qr Query results. + * @param qry Query. + * @param segmentId Index segment ID. + * @param pageSize Page size. + * @param removeMapping Remove mapping flag. + */ + @SuppressWarnings("unchecked") + private void sendNextPage(MapNodeResults nodeRess, ClusterNode node, MapQueryResults qr, int qry, int segmentId, + int pageSize, boolean removeMapping) { + try { + GridQueryNextPageResponse msg = prepareNextPage(nodeRess, node, qr, qry, segmentId, pageSize, removeMapping); + + if (msg != null) { + if (node.isLocal()) + h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg); + else + ctx.io().sendToGridTopic(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL); + } } catch (IgniteCheckedException e) { U.error(log, "Failed to send message.", e); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexIterator.java index 1c0efb39ac6d1..851e1e46dbac0 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexIterator.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexIterator.java @@ -22,11 +22,12 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; import org.h2.index.Cursor; import org.h2.result.Row; +import org.jetbrains.annotations.Nullable; /** * Iterator that transparently and sequentially traverses a bunch of {@link GridMergeIndex} objects. @@ -59,6 +60,9 @@ class GridMergeIndexIterator implements Iterator>, AutoCloseable { /** Whether remote resources were released. */ private boolean released; + /** */ + private MvccQueryTracker mvccTracker; + /** * Constructor. * @@ -69,14 +73,19 @@ class GridMergeIndexIterator implements Iterator>, AutoCloseable { * @param distributedJoins Distributed joins. * @throws IgniteCheckedException if failed. */ - GridMergeIndexIterator(GridReduceQueryExecutor rdcExec, Collection nodes, ReduceQueryRun run, - long qryReqId, boolean distributedJoins) + GridMergeIndexIterator(GridReduceQueryExecutor rdcExec, + Collection nodes, + ReduceQueryRun run, + long qryReqId, + boolean distributedJoins, + @Nullable MvccQueryTracker mvccTracker) throws IgniteCheckedException { this.rdcExec = rdcExec; this.nodes = nodes; this.run = run; this.qryReqId = qryReqId; this.distributedJoins = distributedJoins; + this.mvccTracker = mvccTracker; this.idxIter = run.indexes().iterator(); @@ -155,7 +164,7 @@ private void advance() { private void releaseIfNeeded() { if (!released) { try { - rdcExec.releaseRemoteResources(nodes, run, qryReqId, distributedJoins); + rdcExec.releaseRemoteResources(nodes, run, qryReqId, distributedJoins, mvccTracker); } finally { released = true; 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 2e23a7e0abac0..910ad1a881bec 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 @@ -37,6 +37,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -60,6 +61,11 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxSelectForUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.TxTopologyVersionFuture; +import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; @@ -67,6 +73,7 @@ import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; +import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.h2.H2FieldsIterator; import org.apache.ignite.internal.processors.query.h2.H2Utils; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; @@ -81,15 +88,18 @@ import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2DmlRequest; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2DmlResponse; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2SelectForUpdateTxDetails; import org.apache.ignite.internal.util.GridIntIterator; import org.apache.ignite.internal.util.GridIntList; import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.typedef.C2; import org.apache.ignite.internal.util.typedef.CIX2; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiClosure; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.transactions.TransactionException; import org.h2.command.ddl.CreateTableData; @@ -104,6 +114,7 @@ import static java.util.Collections.singletonList; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.*; import static org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery.EMPTY_PARAMS; import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.REDUCE; @@ -281,7 +292,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { if (failCode == GridQueryFailResponse.CANCELLED_BY_ORIGINATOR) e.addSuppressed(new QueryCancelledException()); - r.state(msg, e, nodeId); + r.state(e, nodeId); } } @@ -289,7 +300,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { * @param node Node. * @param msg Message. */ - private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse msg) { + private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { final long qryReqId = msg.queryRequestId(); final int qry = msg.query(); final int seg = msg.segmentId(); @@ -308,15 +319,15 @@ private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse try { page = new GridResultPage(ctx, node.id(), msg) { @Override public void fetchNextPage() { + Object errState = r.state(); - if (r.hasError()) { - CacheException err0 = r.cacheException(); + if (errState != null) { + CacheException err0 = errState instanceof CacheException ? (CacheException)errState : null; if (err0 != null && err0.getCause() instanceof IgniteClientDisconnectedException) throw err0; - CacheException e = new CacheException( - (msg.retryCause()!=null) ? msg.retryCause() : "Failed to fetch data from node: " + node.id()); + CacheException e = new CacheException("Failed to fetch data from node: " + node.id()); if (err0 != null) e.addSuppressed(err0); @@ -349,9 +360,16 @@ private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse idx.addPage(page); if (msg.retry() != null) - r.stateWithMessage(msg, node.id()); - else if (msg.page() == 0) // Do count down on each first page received. + retry(r, msg.retry(), node.id()); + else if (msg.page() == 0) { + // Do count down on each first page received. r.latch().countDown(); + + GridNearTxSelectForUpdateFuture sfuFut = r.selectForUpdateFuture(); + + if (sfuFut != null) + sfuFut.onResult(node.id(), (long)msg.allRows(), msg.removeMapping(), null); + } } /** @@ -360,7 +378,7 @@ else if (msg.page() == 0) // Do count down on each first page received. * @param nodeId Node ID. */ private void retry(ReduceQueryRun r, AffinityTopologyVersion retryVer, UUID nodeId) { - r.state("Node left the grid", retryVer, nodeId); + r.state(retryVer, nodeId); } /** @@ -369,10 +387,6 @@ private void retry(ReduceQueryRun r, AffinityTopologyVersion retryVer, UUID node */ private boolean isPreloadingActive(List cacheIds) { for (Integer cacheId : cacheIds) { - - if (null == cacheContext(cacheId)) - throw new CacheException(String.format("Cache not found on local node [cacheId=%d]",cacheId)); - if (hasMovingPartitions(cacheContext(cacheId))) return true; } @@ -385,7 +399,6 @@ private boolean isPreloadingActive(List cacheIds) { * @return {@code True} If cache has partitions in {@link GridDhtPartitionState#MOVING} state. */ private boolean hasMovingPartitions(GridCacheContext cctx) { - assert cctx!=null; return !cctx.isLocal() && cctx.topology().hasMovingPartitions(); } @@ -555,6 +568,7 @@ private Map stableDataNodes(boolean isReplicatedOnly, Aff * @param params Query parameters. * @param parts Partitions. * @param lazy Lazy execution flag. + * @param mvccTracker Query tracker. * @return Rows iterator. */ public Iterator> query( @@ -566,8 +580,10 @@ public Iterator> query( GridQueryCancel cancel, Object[] params, final int[] parts, - boolean lazy - ) { + boolean lazy, + MvccQueryTracker mvccTracker) { + assert !qry.mvccEnabled() || mvccTracker != null; + if (F.isEmpty(params)) params = EMPTY_PARAMS; @@ -577,13 +593,10 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); - ReduceQueryRun lastRun = null; - for (int attempt = 0;; attempt++) { - if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) { - String rcValue = lastRun.rootCause(); - throw new CacheException((!F.isEmpty(rcValue))?rcValue:("Failed to map SQL query to topology.")); - } + if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) + throw new CacheException("Failed to map SQL query to topology."); + if (attempt != 0) { try { Thread.sleep(attempt * 10); // Wait for exchange. @@ -597,19 +610,52 @@ public Iterator> query( long qryReqId = qryIdGen.incrementAndGet(); - final ReduceQueryRun r = new ReduceQueryRun(qryReqId, qry.originalSql(), schemaName, - h2.connectionForSchema(schemaName), qry.mapQueries().size(), qry.pageSize(), - U.currentTimeMillis(), cancel); + List cacheIds = qry.cacheIds(); + + boolean mvccEnabled = mvccEnabled(ctx); + + final GridNearTxLocal curTx = mvccEnabled ? checkActive(tx(ctx)) : null; + + final GridNearTxSelectForUpdateFuture sfuFut; + + final boolean clientFirst; + + AffinityTopologyVersion topVer; + + if (qry.forUpdate()) { + // Indexing should have started TX at this point for FOR UPDATE query. + assert mvccEnabled && curTx != null; + + try { + TxTopologyVersionFuture topFut = new TxTopologyVersionFuture(curTx, mvccTracker.context()); + + topVer = topFut.get(); + + clientFirst = topFut.clientFirst(); + } + catch (IgniteCheckedException e) { + throw new IgniteSQLException("Failed to map SELECT FOR UPDATE query on topology.", e); + } - AffinityTopologyVersion topVer = h2.readyTopologyVersion(); + sfuFut = new GridNearTxSelectForUpdateFuture(mvccTracker.context(), curTx, timeoutMillis); + } + else { + sfuFut = null; + + clientFirst = false; + + topVer = h2.readyTopologyVersion(); - // Check if topology is changed while retrying on locked topology. - if (h2.serverTopologyChanged(topVer) && ctx.cache().context().lockedTopologyVersion(null) != null) { - throw new CacheException(new TransactionException("Server topology is changed during query " + - "execution inside a transaction. It's recommended to rollback and retry transaction.")); + // Check if topology has changed while retrying on locked topology. + if (h2.serverTopologyChanged(topVer) && ctx.cache().context().lockedTopologyVersion(null) != null) { + throw new CacheException(new TransactionException("Server topology is changed during query " + + "execution inside a transaction. It's recommended to rollback and retry transaction.")); + } } - List cacheIds = qry.cacheIds(); + final ReduceQueryRun r = new ReduceQueryRun(qryReqId, qry.originalSql(), schemaName, + h2.connectionForSchema(schemaName), qry.mapQueries().size(), qry.pageSize(), + U.currentTimeMillis(), sfuFut, cancel); Collection nodes; @@ -645,8 +691,12 @@ public Iterator> query( partsMap = nodesParts.partitionsMap(); qryMap = nodesParts.queryPartitionsMap(); - if (nodes == null) + if (nodes == null) { + if (sfuFut != null) + sfuFut.onDone(0L, null); + continue; // Retry. + } assert !nodes.isEmpty(); @@ -664,6 +714,9 @@ public Iterator> query( } } + if (sfuFut != null && !sfuFut.isFailed()) + sfuFut.init(topVer, nodes); + int tblIdx = 0; final boolean skipMergeTbl = !qry.explain() && qry.skipMergeTable(); @@ -751,6 +804,24 @@ public Iterator> query( boolean retry = false; + // Always enforce join order on map side to have consistent behavior. + int flags = GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER; + + if (distributedJoins) + flags |= GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS; + + if (qry.isLocal()) + flags |= GridH2QueryRequest.FLAG_IS_LOCAL; + + if (qry.explain()) + flags |= GridH2QueryRequest.FLAG_EXPLAIN; + + if (isReplicatedOnly) + flags |= GridH2QueryRequest.FLAG_REPLICATED; + + if (lazy && mapQrys.size() == 1) + flags |= GridH2QueryRequest.FLAG_LAZY; + GridH2QueryRequest req = new GridH2QueryRequest() .requestId(qryReqId) .topologyVersion(topVer) @@ -760,14 +831,77 @@ public Iterator> query( .partitions(convert(partsMap)) .queries(mapQrys) .parameters(params) - .flags(prepareFlags(qry, lazy, mapQrys.size())) + .flags(flags) .timeout(timeoutMillis) .schemaName(schemaName); - if (send(nodes, req, parts == null ? null : new ExplicitPartitionsSpecializer(qryMap), false)) { + if (curTx != null && curTx.mvccSnapshot() != null) + req.mvccSnapshot(curTx.mvccSnapshot()); + else if (mvccTracker != null) + req.mvccSnapshot(mvccTracker.snapshot()); + + final C2 pspec = + (parts == null ? null : new ExplicitPartitionsSpecializer(qryMap)); + + final C2 spec; + + if (qry.forUpdate()) { + final AtomicInteger cnt = new AtomicInteger(); + + spec = new C2() { + @Override public Message apply(ClusterNode clusterNode, Message msg) { + assert msg instanceof GridH2QueryRequest; + + GridH2QueryRequest res = pspec != null ? (GridH2QueryRequest)pspec.apply(clusterNode, msg) : + new GridH2QueryRequest((GridH2QueryRequest)msg); + + GridH2SelectForUpdateTxDetails txReq = new GridH2SelectForUpdateTxDetails( + curTx.threadId(), + IgniteUuid.randomUuid(), + cnt.incrementAndGet(), + curTx.subjectId(), + curTx.xidVersion(), + curTx.taskNameHash(), + clientFirst, + curTx.remainingTime()); + + res.txDetails(txReq); + + return res; + } + }; + } + else + spec = pspec; + + if (send(nodes, req, spec, false)) { awaitAllReplies(r, nodes, cancel); - retry = analyseCurrentRun(r); + Object state = r.state(); + + if (state != null) { + if (state instanceof CacheException) { + CacheException err = (CacheException)state; + + if (err.getCause() instanceof IgniteClientDisconnectedException) + throw err; + + if (wasCancelled(err)) + throw new QueryCancelledException(); // Throw correct exception. + + throw new CacheException("Failed to run map query remotely." + err.getMessage(), err); + } + + if (state instanceof AffinityTopologyVersion) { + retry = true; + + // On-the-fly topology change must not be possible in FOR UPDATE case. + assert sfuFut == null; + + // If remote node asks us to retry then we have outdated full partition map. + h2.awaitForReadyTopologyVersion((AffinityTopologyVersion)state); + } + } } else // Send failed. retry = true; @@ -776,7 +910,12 @@ public Iterator> query( if (!retry) { if (skipMergeTbl) { - resIter = new GridMergeIndexIterator(this, finalNodes, r, qryReqId, qry.distributedJoins()); + resIter = new GridMergeIndexIterator(this, + finalNodes, + r, + qryReqId, + qry.distributedJoins(), + mvccTracker); release = false; } @@ -803,24 +942,28 @@ public Iterator> query( timeoutMillis, cancel); - resIter = new H2FieldsIterator(res); + resIter = new H2FieldsIterator(res, mvccTracker, false); + + mvccTracker = null; // To prevent callback inside finally block; } finally { GridH2QueryContext.clearThreadLocal(); } } } - - if (retry) { - assert r != null; - lastRun=r; - + else { if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); + if (sfuFut != null) + sfuFut.onDone(0L); + continue; } + if (sfuFut != null) + sfuFut.get(); + return new GridQueryCacheObjectsIterator(resIter, h2.objectContext(), keepBinary); } catch (IgniteCheckedException | RuntimeException e) { @@ -828,11 +971,21 @@ public Iterator> query( U.closeQuiet(r.connection()); + CacheException resEx = null; + if (e instanceof CacheException) { if (wasCancelled((CacheException)e)) - throw new CacheException("Failed to run reduce query locally.", new QueryCancelledException()); + resEx = new CacheException("Failed to run reduce query locally.", + new QueryCancelledException()); + else + resEx = (CacheException)e; + } + + if (resEx != null) { + if (sfuFut != null) + sfuFut.onDone(resEx); - throw (CacheException)e; + throw resEx; } Throwable cause = e; @@ -841,23 +994,20 @@ public Iterator> query( Throwable disconnectedErr = ((IgniteCheckedException)e).getCause(IgniteClientDisconnectedException.class); - if ( QueryCancelledException.class.isAssignableFrom(e.getClass()) ) - cause = new QueryCancelledException(String.format( - "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s]", - qry.originalSql(), - ctx.localNodeId(), - "Cancelled by client" - )); - if (disconnectedErr != null) cause = disconnectedErr; } - throw new CacheException("Failed to run reduce query locally.", cause); + resEx = new CacheException("Failed to run reduce query locally.", cause); + + if (sfuFut != null) + sfuFut.onDone(resEx); + + throw resEx; } finally { if (release) { - releaseRemoteResources(finalNodes, r, qryReqId, qry.distributedJoins()); + releaseRemoteResources(finalNodes, r, qryReqId, qry.distributedJoins(), mvccTracker); if (!skipMergeTbl) { for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++) @@ -868,64 +1018,6 @@ public Iterator> query( } } - /** - * Analyse reduce query run to decide if retry is required - * @param r reduce query run to be analysed - * @return true if retry is required, false otherwise - * @throws IgniteCheckedException in case of reduce query run contains exception record - */ - private boolean analyseCurrentRun(ReduceQueryRun r) throws IgniteCheckedException { - if (r.hasError()) { - if (r.cacheException() != null) { - CacheException err = r.cacheException(); - - if (err.getCause() instanceof IgniteClientDisconnectedException) - throw err; - - Exception cause = wasCancelled(err) || X.hasCause(err, QueryCancelledException.class) - ? new QueryCancelledException(r.rootCause()) - : err; - - throw new CacheException("Failed to run map query remotely." + cause.getMessage(), cause); - } else { - // If remote node asks us to retry then we have outdated full partition map. - h2.awaitForReadyTopologyVersion(r.topVersion()); - - return true; - } - } - return false; - } - - /** - * Builds flag out of parameters - * @param qry query parameter holder - * @param lazy if lazy execution - * @param mapQrysSize number of queries - * @return flag - */ - private int prepareFlags(GridCacheTwoStepQuery qry, boolean lazy, int mapQrysSize) { - // Always enforce join order on map side to have consistent behavior. - int flags = GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER; - - if (qry.distributedJoins()) - flags |= GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS; - - if (qry.isLocal()) - flags |= GridH2QueryRequest.FLAG_IS_LOCAL; - - if (qry.explain()) - flags |= GridH2QueryRequest.FLAG_EXPLAIN; - - if (qry.isReplicatedOnly()) - flags |= GridH2QueryRequest.FLAG_REPLICATED; - - if (lazy && mapQrysSize == 1) - flags |= GridH2QueryRequest.FLAG_LAZY; - - return flags; - } - /** * * @param schemaName Schema name. @@ -1110,7 +1202,10 @@ private boolean wasCancelled(CacheException e) { * @param distributedJoins Distributed join flag. */ public void releaseRemoteResources(Collection nodes, ReduceQueryRun r, long qryReqId, - boolean distributedJoins) { + boolean distributedJoins, MvccQueryTracker mvccTracker) { + if (mvccTracker != null) + mvccTracker.onDone(); + // For distributedJoins need always send cancel request to cleanup resources. if (distributedJoins) send(nodes, new GridQueryCancelRequest(qryReqId), null, false); @@ -1537,7 +1632,7 @@ private String getPlan(ResultSet rs) throws IgniteCheckedException { * @param runLocParallel Run local handler in parallel thread. * @return {@code true} If all messages sent successfully. */ - private boolean send( + public boolean send( Collection nodes, Message msg, @Nullable IgniteBiClosure specialize, @@ -1805,7 +1900,7 @@ private static long retryTimeout(long qryTimeout) { } /** */ - private static class ExplicitPartitionsSpecializer implements IgniteBiClosure { + private static class ExplicitPartitionsSpecializer implements C2 { /** Partitions map. */ private final Map partsMap; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java index 103084e662052..3c176407f98f0 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.query.h2.twostep; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Iterator; @@ -26,7 +28,9 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.plugin.extensions.communication.Message; import org.h2.value.Value; @@ -68,9 +72,25 @@ public GridResultPage(final GridKernalContext ctx, UUID src, GridQueryNextPageRe Collection plainRows = res.plainRows(); if (plainRows != null) { + assert plainRows instanceof ArrayList; + rowsInPage = plainRows.size(); - rows = (Iterator)plainRows.iterator(); + if (rowsInPage == 0 || ((ArrayList)plainRows).get(0).length == res.columns()) + rows = (Iterator)plainRows.iterator(); + else { + // If it's a result of SELECT FOR UPDATE (we can tell by difference in number + // of columns checked above), we need to strip off stuff we don't need. + rows = F.iterator(plainRows, new IgniteClosure() { + @Override public Value[] apply(Object o) { + Value[] row = (Value[])o; + + assert row.length >= res.columns(); + + return Arrays.copyOfRange(row, 0, res.columns()); + } + }, true); + } } else { final int cols = res.columns(); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java index 45f9c1fb89340..76527bc6d6ed6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java @@ -51,17 +51,22 @@ class MapQueryResults { /** */ private volatile boolean cancelled; + /** {@code SELECT FOR UPDATE} flag. */ + private final boolean forUpdate; + /** * Constructor. - * + * @param h2 Indexing instance. * @param qryReqId Query request ID. * @param qrys Number of queries. * @param cctx Cache context. * @param lazyWorker Lazy worker (if any). + * @param forUpdate {@code SELECT FOR UPDATE} flag. */ @SuppressWarnings("unchecked") MapQueryResults(IgniteH2Indexing h2, long qryReqId, int qrys, @Nullable GridCacheContext cctx, - @Nullable MapQueryLazyWorker lazyWorker) { + @Nullable MapQueryLazyWorker lazyWorker, boolean forUpdate) { + this.forUpdate = forUpdate; this.h2 = h2; this.qryReqId = qryReqId; this.cctx = cctx; @@ -101,11 +106,11 @@ MapQueryLazyWorker lazyWorker() { /** * Add result. - * * @param qry Query result index. * @param q Query object. * @param qrySrcNodeId Query source node. * @param rs Result set. + * @param params Query arguments. */ void addResult(int qry, GridCacheSqlQuery q, UUID qrySrcNodeId, ResultSet rs, Object[] params) { MapQueryResult res = new MapQueryResult(h2, rs, cctx, qrySrcNodeId, q, params, lazyWorker); @@ -172,4 +177,11 @@ boolean cancelled() { long queryRequestId() { return qryReqId; } + + /** + * @return {@code SELECT FOR UPDATE} flag. + */ + public boolean isForUpdate() { + return forUpdate; + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 42897df265ed3..df72e8cde583c 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -25,10 +25,10 @@ import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxSelectForUpdateFuture; import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse; import org.h2.jdbc.JdbcConnection; import org.jetbrains.annotations.Nullable; @@ -53,12 +53,14 @@ class ReduceQueryRun { /** */ private final int pageSize; - /** */ - private final AtomicReference state = new AtomicReference<>(); + /** Can be either CacheException in case of error or AffinityTopologyVersion to retry if needed. */ + private final AtomicReference state = new AtomicReference<>(); + + /** Future controlling {@code SELECT FOR UPDATE} query execution. */ + private final GridNearTxSelectForUpdateFuture selectForUpdateFut; /** * Constructor. - * * @param id Query ID. * @param qry Query text. * @param schemaName Schema name. @@ -66,67 +68,48 @@ class ReduceQueryRun { * @param idxsCnt Number of indexes. * @param pageSize Page size. * @param startTime Start time. + * @param selectForUpdateFut Future controlling {@code SELECT FOR UPDATE} query execution. * @param cancel Query cancel handler. */ ReduceQueryRun(Long id, String qry, String schemaName, Connection conn, int idxsCnt, int pageSize, long startTime, - GridQueryCancel cancel) { - this.qry = new GridRunningQueryInfo(id, qry, SQL_FIELDS, schemaName, startTime, cancel, false); + GridNearTxSelectForUpdateFuture selectForUpdateFut, GridQueryCancel cancel) { + this.qry = new GridRunningQueryInfo(id, qry, SQL_FIELDS, schemaName, startTime, cancel, + false); this.conn = (JdbcConnection)conn; this.idxs = new ArrayList<>(idxsCnt); this.pageSize = pageSize > 0 ? pageSize : GridCacheTwoStepQuery.DFLT_PAGE_SIZE; + + this.selectForUpdateFut = selectForUpdateFut; } /** * @param o Fail state object. * @param nodeId Node ID. */ - void state(String msg, Object o, @Nullable UUID nodeId) { + void state(Object o, @Nullable UUID nodeId) { assert o != null; - assert o instanceof CacheException || o instanceof AffinityTopologyVersion : o.getClass(); - if ( o instanceof CacheException ) - state(new State((CacheException)o, msg, null, nodeId)); - - else - state(new State(null, msg, (AffinityTopologyVersion)o, nodeId)); - } - - /** - * @param msg corresponding response message - * @param nodeId Node ID. - */ - void stateWithMessage(GridQueryNextPageResponse msg, @Nullable UUID nodeId) { - assert msg != null; - - assert msg.retry() != null; - - state(new State(null, msg.retryCause(), msg.retry(), nodeId)); - } - - /** - * - * @param state state - */ - private void state(State state){ - if (!this.state.compareAndSet(null, state)) + if (!state.compareAndSet(null, o)) return; while (latch.getCount() != 0) // We don't need to wait for all nodes to reply. latch.countDown(); + CacheException e = o instanceof CacheException ? (CacheException) o : null; + for (GridMergeIndex idx : idxs) // Fail all merge indexes. - idx.fail(state.nodeId, state.ex); + idx.fail(nodeId, e); } /** * @param e Error. */ void disconnected(CacheException e) { - state("", e, null); + state(e, null); } /** @@ -150,31 +133,13 @@ JdbcConnection connection() { return conn; } - /** */ - boolean hasError(){ - return state.get()!=null; - } - - /** */ - CacheException cacheException() { - State st = state.get(); - - return st!=null ? st.ex : null; - } - - /** */ - AffinityTopologyVersion topVersion(){ - State st = state.get(); - - return st!=null ? st.topVer : null; + /** + * @return State. + */ + Object state() { + return state.get(); } - /** */ - String rootCause(){ - State st = state.get(); - - return st!=null ? st.rootCause : null; - } /** * @return Indexes. */ @@ -196,30 +161,10 @@ void latch(CountDownLatch latch) { this.latch = latch; } - /** */ - private static class State{ - - /** */ - private final CacheException ex; - - /** */ - private final String rootCause; - - /** */ - private final AffinityTopologyVersion topVer; - - /** */ - private final UUID nodeId; - - /** */ - private State(CacheException ex, String rootCause, AffinityTopologyVersion topVer, UUID nodeId){ - this.ex=ex; - - this.rootCause = rootCause; - - this.topVer = topVer; - - this.nodeId = nodeId; - } + /** + * @return {@code SELECT FOR UPDATE} future, if any. + */ + @Nullable public GridNearTxSelectForUpdateFuture selectForUpdateFuture() { + return selectForUpdateFut; } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java index 4e1fadbfe3825..86721949705c3 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; import org.apache.ignite.internal.processors.cache.query.QueryTable; @@ -42,6 +43,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery.EMPTY_PARAMS; @@ -133,6 +135,12 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable { /** Schema name. */ private String schemaName; + /** */ + private MvccSnapshot mvccSnapshot; + + /** TX details holder for {@code SELECT FOR UPDATE}, or {@code null} if not applicable. */ + private GridH2SelectForUpdateTxDetails txReq; + /** * Required by {@link Externalizable} */ @@ -157,6 +165,25 @@ public GridH2QueryRequest(GridH2QueryRequest req) { params = req.params; paramsBytes = req.paramsBytes; schemaName = req.schemaName; + mvccSnapshot = req.mvccSnapshot; + txReq = req.txReq; + } + + /** + * @return MVCC snapshot. + */ + @Nullable public MvccSnapshot mvccSnapshot() { + return mvccSnapshot; + } + + /** + * @param mvccSnapshot MVCC snapshot version. + * @return {@code this}. + */ + public GridH2QueryRequest mvccSnapshot(MvccSnapshot mvccSnapshot) { + this.mvccSnapshot = mvccSnapshot; + + return this; } /** @@ -373,6 +400,20 @@ public GridH2QueryRequest schemaName(String schemaName) { return this; } + /** + * @return TX details holder for {@code SELECT FOR UPDATE}, or {@code null} if not applicable. + */ + public GridH2SelectForUpdateTxDetails txDetails() { + return txReq; + } + + /** + * @param txDetails TX details holder for {@code SELECT FOR UPDATE}, or {@code null} if not applicable. + */ + public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { + this.txReq = txDetails; + } + /** {@inheritDoc} */ @Override public void marshall(Marshaller m) { if (paramsBytes != null) @@ -435,65 +476,77 @@ public GridH2QueryRequest schemaName(String schemaName) { writer.incrementState(); case 2: - if (!writer.writeInt("pageSize", pageSize)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 3: - if (!writer.writeByteArray("paramsBytes", paramsBytes)) + if (!writer.writeInt("pageSize", pageSize)) return false; writer.incrementState(); case 4: - if (!writer.writeMap("parts", parts, MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR)) + if (!writer.writeByteArray("paramsBytes", paramsBytes)) return false; writer.incrementState(); case 5: - if (!writer.writeCollection("qrys", qrys, MessageCollectionItemType.MSG)) + if (!writer.writeMap("parts", parts, MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR)) return false; writer.incrementState(); case 6: - if (!writer.writeLong("reqId", reqId)) + if (!writer.writeIntArray("qryParts", qryParts)) return false; writer.incrementState(); case 7: - if (!writer.writeCollection("tbls", tbls, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("qrys", qrys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 8: - if (!writer.writeInt("timeout", timeout)) + if (!writer.writeLong("reqId", reqId)) return false; writer.incrementState(); case 9: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeString("schemaName", schemaName)) return false; writer.incrementState(); - case 10: - if (!writer.writeIntArray("qryParts", qryParts)) + if (!writer.writeCollection("tbls", tbls, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 11: - if (!writer.writeString("schemaName", schemaName)) + if (!writer.writeInt("timeout", timeout)) + return false; + + writer.incrementState(); + + case 12: + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); + + case 13: + if (!writer.writeMessage("txReq", txReq)) + return false; + + writer.incrementState(); + } return true; @@ -524,7 +577,7 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 2: - pageSize = reader.readInt("pageSize"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -532,7 +585,7 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 3: - paramsBytes = reader.readByteArray("paramsBytes"); + pageSize = reader.readInt("pageSize"); if (!reader.isLastRead()) return false; @@ -540,7 +593,7 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 4: - parts = reader.readMap("parts", MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR, false); + paramsBytes = reader.readByteArray("paramsBytes"); if (!reader.isLastRead()) return false; @@ -548,7 +601,7 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 5: - qrys = reader.readCollection("qrys", MessageCollectionItemType.MSG); + parts = reader.readMap("parts", MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR, false); if (!reader.isLastRead()) return false; @@ -556,7 +609,7 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 6: - reqId = reader.readLong("reqId"); + qryParts = reader.readIntArray("qryParts"); if (!reader.isLastRead()) return false; @@ -564,7 +617,7 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 7: - tbls = reader.readCollection("tbls", MessageCollectionItemType.MSG); + qrys = reader.readCollection("qrys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -572,7 +625,7 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 8: - timeout = reader.readInt("timeout"); + reqId = reader.readLong("reqId"); if (!reader.isLastRead()) return false; @@ -580,16 +633,15 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 9: - topVer = reader.readMessage("topVer"); + schemaName = reader.readString("schemaName"); if (!reader.isLastRead()) return false; reader.incrementState(); - case 10: - qryParts = reader.readIntArray("qryParts"); + tbls = reader.readCollection("tbls", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -597,12 +649,29 @@ public GridH2QueryRequest schemaName(String schemaName) { reader.incrementState(); case 11: - schemaName = reader.readString("schemaName"); + timeout = reader.readInt("timeout"); if (!reader.isLastRead()) return false; reader.incrementState(); + + case 12: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 13: + txReq = reader.readMessage("txReq"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridH2QueryRequest.class); @@ -615,7 +684,7 @@ public GridH2QueryRequest schemaName(String schemaName) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 12; + return 14; } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2SelectForUpdateTxDetails.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2SelectForUpdateTxDetails.java new file mode 100644 index 0000000000000..a13a9c11abe81 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2SelectForUpdateTxDetails.java @@ -0,0 +1,299 @@ +/* + * 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.twostep.msg; + +import java.nio.ByteBuffer; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * TX details holder for {@link GridH2QueryRequest}. + */ +public class GridH2SelectForUpdateTxDetails implements Message { + /** */ + private static final long serialVersionUID = 8166491041528984454L; + /** */ + private long threadId; + + /** */ + private IgniteUuid futId; + + /** */ + private int miniId; + + /** */ + private UUID subjId; + + /** */ + private GridCacheVersion lockVer; + + /** */ + private int taskNameHash; + + /** */ + private boolean clientFirst; + + /** */ + private long timeout; + + /** + * Default constructor. + */ + GridH2SelectForUpdateTxDetails() { + // No-op. + } + + /** + * @param threadId Thread id. + * @param futId Future id. + * @param miniId Mini fture id. + * @param subjId Subject id. + * @param lockVer Lock version. + * @param taskNameHash Task name hash. + * @param clientFirst {@code True} if this is the first client request. + * @param timeout Tx timeout. + */ + public GridH2SelectForUpdateTxDetails(long threadId, IgniteUuid futId, int miniId, UUID subjId, + GridCacheVersion lockVer, int taskNameHash, boolean clientFirst, long timeout) { + this.threadId = threadId; + this.futId = futId; + this.miniId = miniId; + this.subjId = subjId; + this.lockVer = lockVer; + this.taskNameHash = taskNameHash; + this.clientFirst = clientFirst; + this.timeout = timeout; + } + + /** + * @return Thread id. + */ + public long threadId() { + return threadId; + } + + /** + * @return Future id. + */ + public IgniteUuid futureId() { + return futId; + } + + /** + * @return Mini fture id. + */ + public int miniId() { + return miniId; + } + + /** + * @return Subject id. + */ + public UUID subjectId() { + return subjId; + } + + /** + * @return Lock version. + */ + public GridCacheVersion version() { + return lockVer; + } + + /** + * @return Task name hash. + */ + public int taskNameHash() { + return taskNameHash; + } + + /** + * @return {@code True} if this is the first client request in transaction. + */ + public boolean firstClientRequest() { + return clientFirst; + } + + /** + * @return Tx timeout. + */ + public long timeout() { + return timeout; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeBoolean("clientFirst", clientFirst)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeIgniteUuid("futId", futId)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeMessage("lockVer", lockVer)) + return false; + + writer.incrementState(); + + case 3: + if (!writer.writeInt("miniId", miniId)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeUuid("subjId", subjId)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeLong("threadId", threadId)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeLong("timeout", timeout)) + 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: + clientFirst = reader.readBoolean("clientFirst"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + futId = reader.readIgniteUuid("futId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + lockVer = reader.readMessage("lockVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 3: + miniId = reader.readInt("miniId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + subjId = reader.readUuid("subjId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + threadId = reader.readLong("threadId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + timeout = reader.readLong("timeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridH2SelectForUpdateTxDetails.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return -57; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 8; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java index 3c133928ef7c7..c399d7614a34f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java @@ -118,6 +118,9 @@ public class GridH2ValueMessageFactory implements MessageFactory { case -56: return new GridH2DmlResponse(); + + case -57: + return new GridH2SelectForUpdateTxDetails(); } return null; @@ -126,14 +129,17 @@ public class GridH2ValueMessageFactory implements MessageFactory { /** * @param src Source values. * @param dst Destination collection. + * @param cnt Number of columns to actually send. * @return Destination collection. * @throws IgniteCheckedException If failed. */ - public static Collection toMessages(Collection src, Collection dst) + public static Collection toMessages(Collection src, Collection dst, int cnt) throws IgniteCheckedException { for (Value[] row : src) { - for (Value val : row) - dst.add(toMessage(val)); + assert row.length >= cnt; + + for (int i = 0; i < cnt; i++) + dst.add(toMessage(row[i])); } return dst; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTest.java new file mode 100644 index 0000000000000..6652559a15060 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTest.java @@ -0,0 +1,335 @@ +/* + * 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.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.FieldsQueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.util.typedef.F; +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.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.apache.ignite.transactions.TransactionState; + +/** + * + */ +public class DdlTransactionSelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setTransactionConfiguration(new TransactionConfiguration() + .setDefaultTxIsolation(TransactionIsolation.REPEATABLE_READ) + .setDefaultTxConcurrency(TransactionConcurrency.PESSIMISTIC) + .setDefaultTxTimeout(5000)); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(discoSpi); + cfg.setCacheConfiguration(getCacheConfiguration()); + cfg.setClientMode(client); + + return cfg; + } + + /** + * @return Cache configuration. + */ + private CacheConfiguration getCacheConfiguration() { + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setNearConfiguration(null); + + return ccfg; + } + + /** + * @throws Exception If failed. + */ + public void testTxIsCommittedOnDdlRequestMultinodeClient() throws Exception { + startGridsMultiThreaded(4, false); + + client = true; + + Ignite node = startGrid(4); + + awaitPartitionMapExchange(); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = node.transactions().txStart()) { + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery( + "CREATE TABLE " + + " person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" + + "WITH " + + " \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) { + + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(1, rows.size()); + + assertEquals(0L, rows.get(0).get(0)); + } + + assertTrue(tx.state() == TransactionState.COMMITTED); + } + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) { + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(0, rows.size()); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testTxIsCommittedOnDdlRequestMultinode() throws Exception { + Ignite node = startGridsMultiThreaded(4); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = node.transactions().txStart()) { + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery( + "CREATE TABLE " + + " person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" + + "WITH " + + " \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) { + + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(1, rows.size()); + + assertEquals(0L, rows.get(0).get(0)); + } + + assertTrue(tx.state() == TransactionState.COMMITTED); + } + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) { + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(0, rows.size()); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testTxIsCommittedOnDdlRequest() throws Exception { + Ignite node = startGrid(); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = node.transactions().txStart()) { + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery( + "CREATE TABLE " + + " person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" + + "WITH " + + " \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) { + + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(1, rows.size()); + + assertEquals(0L, rows.get(0).get(0)); + } + + assertTrue(tx.state() == TransactionState.COMMITTED); + } + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) { + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(0, rows.size()); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testDdlRequestWithoutTxMultinodeClient() throws Exception { + startGridsMultiThreaded(4, false); + + client = true; + + Ignite node = startGrid(4); + + awaitPartitionMapExchange(); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery( + "CREATE TABLE " + + " person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" + + "WITH " + + " \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) { + + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(1, rows.size()); + + assertEquals(0L, rows.get(0).get(0)); + } + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) { + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(0, rows.size()); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testDdlRequestWithoutTxMultinode() throws Exception { + Ignite node = startGridsMultiThreaded(4); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery( + "CREATE TABLE " + + " person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" + + "WITH " + + " \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) { + + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(1, rows.size()); + + assertEquals(0L, rows.get(0).get(0)); + } + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) { + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(0, rows.size()); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testDdlRequestWithoutTx() throws Exception { + Ignite node = startGrid(); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery( + "CREATE TABLE " + + " person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" + + "WITH " + + " \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) { + + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(1, rows.size()); + + assertEquals(0L, rows.get(0).get(0)); + } + + try (FieldsQueryCursor> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) { + assertNotNull(cur); + + List> rows = cur.getAll(); + + assertEquals(0, rows.size()); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java index 7212bf8ffe309..59be13878aebd 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryH2IndexingLeakTest.java @@ -27,6 +27,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.h2.H2ConnectionWrapper; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.CAX; @@ -120,9 +121,14 @@ protected CacheConfiguration cacheConfiguration() { private static int getStatementCacheSize(GridQueryProcessor qryProcessor) { IgniteH2Indexing h2Idx = GridTestUtils.getFieldValue(qryProcessor, GridQueryProcessor.class, "idx"); - ConcurrentMap stmtCache = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "stmtCache"); + ConcurrentMap conns = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "conns"); - return stmtCache.size(); + int cntr = 0; + + for (H2ConnectionWrapper w : conns.values()) + cntr += w.statementCacheSize(); + + return cntr; } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java index 0a0efc72d373d..7f1e2e74e8ec1 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java @@ -21,12 +21,15 @@ import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; +import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -36,12 +39,17 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor; import org.apache.ignite.internal.processors.port.GridPortRecord; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; +import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl; import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl; import org.apache.ignite.internal.processors.query.QueryUtils; @@ -51,6 +59,9 @@ import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; +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.jetbrains.annotations.Nullable; @@ -59,6 +70,9 @@ */ @SuppressWarnings("unchecked") public abstract class AbstractSchemaSelfTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + /** Cache. */ protected static final String CACHE_NAME = "cache"; @@ -101,6 +115,59 @@ public abstract class AbstractSchemaSelfTest extends GridCommonAbstractTest { /** Field 2 escaped. */ protected static final String FIELD_NAME_2_ESCAPED = "field2"; + /** + * Create common node configuration. + * + * @param idx Index. + * @return Configuration. + * @throws Exception If failed. + */ + protected IgniteConfiguration commonConfiguration(int idx) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(getTestIgniteInstanceName(idx)); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + cfg.setMarshaller(new BinaryMarshaller()); + + DataStorageConfiguration memCfg = new DataStorageConfiguration().setDefaultDataRegionConfiguration( + new DataRegionConfiguration().setMaxSize(128 * 1024 * 1024)); + + cfg.setDataStorageConfiguration(memCfg); + + return optimize(cfg); + } + + /** + * Ensure that SQL exception is thrown. + * + * @param r Runnable. + * @param expCode Error code. + */ + static void assertSqlException(DynamicIndexAbstractBasicSelfTest.RunnableX r, int expCode) { + try { + try { + r.run(); + } + catch (CacheException e) { + if (e.getCause() != null) + throw (Exception)e.getCause(); + else + throw e; + } + } + catch (IgniteSQLException e) { + assertEquals("Unexpected error code [expected=" + expCode + ", actual=" + e.statusCode() + ']', + expCode, e.statusCode()); + + return; + } + catch (Exception e) { + fail("Unexpected exception: " + e); + } + + fail(IgniteSQLException.class.getSimpleName() + " is not thrown."); + } + /** * Get type on the given node for the given cache and table name. Type must exist. * @@ -230,7 +297,7 @@ private static void assertInternalIndexParams(QueryTypeDescriptorImpl typeDesc, * @param node Node to connect to. * @return Thin JDBC connection to specified node. */ - static Connection connect(IgniteEx node) { + public static Connection connect(IgniteEx node) { Collection recs = node.context().ports().records(); GridPortRecord cliLsnrRec = null; @@ -253,6 +320,17 @@ static Connection connect(IgniteEx node) { } } + /** + * @param conn Connection. + * @param sql Statement. + * @throws SQLException if failed. + */ + public static void execute(Connection conn, String sql) throws SQLException { + try (Statement s = conn.createStatement()) { + s.execute(sql); + } + } + /** * Assert index doesn't exist on all nodes. * @@ -347,6 +425,16 @@ protected static QueryIndex index(String name, IgniteBiTuple... return idx; } + /** + * Execute SQL statement on given node. + * + * @param node Node. + * @param sql Statement. + */ + protected List> execute(Ignite node, String sql) { + return queryProcessor(node).querySqlFields(new SqlFieldsQuery(sql).setSchema("PUBLIC"), true).getAll(); + } + /** * Get query processor. * @@ -354,7 +442,17 @@ protected static QueryIndex index(String name, IgniteBiTuple... * @return Query processor. */ static GridQueryProcessor queryProcessor(Ignite node) { - return ((IgniteEx)node).context().query(); + return queryProcessor((IgniteEx)node); + } + + /** + * Get query processor. + * + * @param node Node. + * @return Query processor. + */ + protected static GridQueryProcessor queryProcessor(IgniteEx node) { + return node.context().query(); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java index 3f090620287d2..072f1ab2d136a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java @@ -380,9 +380,9 @@ public void testConcurrentPutRemove() throws Exception { IgniteCache cache = node.cache(CACHE_NAME); if (ThreadLocalRandom.current().nextBoolean()) - cache.put(key(node, key), val(node, val)); + cache.put(key(key), val(node, val)); else - cache.remove(key(node, key)); + cache.remove(key(key)); } return null; @@ -416,7 +416,7 @@ public void testConcurrentPutRemove() throws Exception { IgniteCache cache = srv1.cache(CACHE_NAME).withKeepBinary(); for (int i = 0; i < LARGE_CACHE_SIZE; i++) { - Object key = key(srv1, i); + Object key = key(i); BinaryObject val = cache.get(key); @@ -430,7 +430,7 @@ public void testConcurrentPutRemove() throws Exception { } } - String valTypeName = ((IgniteEx)srv1).context().query().types(CACHE_NAME).iterator().next().valueTypeName(); + String valTypeName = (srv1).context().query().types(CACHE_NAME).iterator().next().valueTypeName(); // Validate query result. for (Ignite node : Ignition.allGrids()) { @@ -469,11 +469,10 @@ private BinaryObject val(Ignite node, int val) { } /** - * @param node Node. * @param id Key. * @return PERSON cache key (int or {@link BinaryObject}). */ - private Object key(Ignite node, int id) { + private Object key(int id) { return id; } @@ -550,7 +549,7 @@ public void checkConcurrentRebalance(boolean addOrRemove) throws Exception { */ private void put(Ignite node, int startIdx, int endIdx) { for (int i = startIdx; i < endIdx; i++) - node.cache(CACHE_NAME).put(key(node, i), val(node, i)); + node.cache(CACHE_NAME).put(key(i), val(node, i)); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java index 4490fc726f91d..b00d750f710c6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java @@ -83,6 +83,13 @@ public abstract class DynamicIndexAbstractSelfTest extends AbstractSchemaSelfTes /** Argument for simple SQL (2). */ protected static final int SQL_ARG_2 = 80; + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + /** * Create server configuration. * diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java index 99661e458b748..7713004446c9f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2ConnectionLeaksSelfTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache.index; -import java.sql.Connection; import java.util.Map; import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; @@ -25,7 +24,6 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.testframework.GridTestUtils; @@ -145,7 +143,7 @@ private void checkConnectionLeaks() throws Exception { boolean notLeak = GridTestUtils.waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { for (int i = 0; i < NODE_CNT; i++) { - Map conns = perThreadConnections(i); + Map conns = perThreadConnections(i); for(Thread t : conns.keySet()) { if (!t.isAlive()) @@ -159,7 +157,7 @@ private void checkConnectionLeaks() throws Exception { if (!notLeak) { for (int i = 0; i < NODE_CNT; i++) { - Map conns = perThreadConnections(i); + Map conns = perThreadConnections(i); for(Thread t : conns.keySet()) log.error("+++ Connection is not closed for thread: " + t.getName()); @@ -173,7 +171,7 @@ private void checkConnectionLeaks() throws Exception { * @param nodeIdx Node index. * @return Per-thread connections. */ - private Map perThreadConnections(int nodeIdx) { + private Map perThreadConnections(int nodeIdx) { return ((IgniteH2Indexing)grid(nodeIdx).context().query().getIndexing()).perThreadConnections(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java index cf47774f70d1d..9579b796d3f6d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; @@ -33,9 +32,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; -import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; -import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.util.typedef.F; /** @@ -297,21 +294,6 @@ private IgniteConfiguration clientConfiguration(int idx) throws Exception { return commonConfiguration(idx).setClientMode(true); } - /** - * Create common node configuration. - * - * @param idx Index. - * @return Configuration. - * @throws Exception If failed. - */ - private IgniteConfiguration commonConfiguration(int idx) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(getTestIgniteInstanceName(idx)); - - cfg.setMarshaller(new BinaryMarshaller()); - - return optimize(cfg); - } - /** * @return Default cache configuration. */ @@ -359,35 +341,4 @@ private CacheConfiguration cacheConfiguration() { * @return Whether to use near cache. */ protected abstract boolean nearCache(); - - /** - * Ensure that SQL exception is thrown. - * - * @param r Runnable. - * @param expCode Error code. - */ - private static void assertSqlException(DynamicIndexAbstractBasicSelfTest.RunnableX r, int expCode) { - try { - try { - r.run(); - } - catch (CacheException e) { - if (e.getCause() != null) - throw (Exception)e.getCause(); - else - throw e; - } - } - catch (IgniteSQLException e) { - assertEquals("Unexpected error code [expected=" + expCode + ", actual=" + e.statusCode() + ']', - expCode, e.statusCode()); - - return; - } - catch (Exception e) { - fail("Unexpected exception: " + e); - } - - fail(IgniteSQLException.class.getSimpleName() + " is not thrown."); - } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java index 82d10cdf31566..6ed914c165937 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java @@ -50,7 +50,6 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; @@ -1609,27 +1608,16 @@ private IgniteConfiguration clientConfiguration(int idx) throws Exception { * @return Configuration. * @throws Exception If failed. */ - private IgniteConfiguration commonConfiguration(int idx) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(getTestIgniteInstanceName(idx)); + protected IgniteConfiguration commonConfiguration(int idx) throws Exception { + IgniteConfiguration cfg = super.commonConfiguration(idx); DataRegionConfiguration dataRegionCfg = new DataRegionConfiguration().setName(DATA_REGION_NAME); - cfg.setMarshaller(new BinaryMarshaller()); cfg.setDataStorageConfiguration(new DataStorageConfiguration().setDataRegionConfigurations(dataRegionCfg)); return optimize(cfg); } - /** - * Execute DDL statement on given node. - * - * @param node Node. - * @param sql Statement. - */ - private List> execute(Ignite node, String sql) { - return queryProcessor(node).querySqlFields(new SqlFieldsQuery(sql).setSchema("PUBLIC"), true).getAll(); - } - /** * Execute DDL statement on given node. * diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java new file mode 100644 index 0000000000000..d2931ba826085 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.index; + +import java.util.concurrent.Callable; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * + */ +public class SqlTransactionsComandsWithMvccDisabledSelfTest extends AbstractSchemaSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(commonConfiguration(0)); + + super.execute(grid(0), "CREATE TABLE INTS(k int primary key, v int) WITH \"wrap_value=false,cache_name=ints," + + "atomicity=transactional\""); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + + /** + * @throws Exception if failed. + */ + public void testBeginWithMvccDisabledThrows() throws Exception { + checkMvccDisabledBehavior("BEGIN"); + } + + /** + * @throws Exception if failed. + */ + public void testCommitWithMvccDisabledThrows() throws Exception { + checkMvccDisabledBehavior("COMMIT"); + } + + /** + * @throws Exception if failed. + */ + public void testRollbackWithMvccDisabledThrows() throws Exception { + checkMvccDisabledBehavior("rollback"); + } + + /** + * @param sql Operation to test. + * @throws Exception if failed. + */ + private void checkMvccDisabledBehavior(String sql) throws Exception { + try (IgniteEx node = startGrid(commonConfiguration(1))) { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + execute(node, sql); + + return null; + } + }, IgniteSQLException.class, "MVCC must be enabled in order to invoke transactional operation: " + sql); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsSelfTest.java new file mode 100644 index 0000000000000..7081e3428e064 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsSelfTest.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.index; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +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.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.GatewayProtectedCacheProxy; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionState; + +/** + * Tests to check behavior regarding transactions started via SQL. + */ +public class SqlTransactionsSelfTest extends AbstractSchemaSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(commonConfiguration(0).setMvccEnabled(true)); + + super.execute(node(), "CREATE TABLE INTS(k int primary key, v int) WITH \"wrap_value=false,cache_name=ints," + + "atomicity=transactional\""); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * Test that BEGIN opens a transaction. + */ + public void testBegin() { + execute(node(), "BEGIN"); + + assertTxPresent(); + + assertTxState(tx(), TransactionState.ACTIVE); + } + + /** + * Test that COMMIT commits a transaction. + */ + public void testCommit() { + execute(node(), "BEGIN WORK"); + + assertTxPresent(); + + Transaction tx = tx(); + + assertTxState(tx, TransactionState.ACTIVE); + + execute(node(), "COMMIT TRANSACTION"); + + assertTxState(tx, TransactionState.COMMITTED); + + assertSqlTxNotPresent(); + } + + /** + * Test that COMMIT without a transaction yields nothing. + */ + public void testCommitNoTransaction() { + execute(node(), "COMMIT"); + } + + /** + * Test that ROLLBACK without a transaction yields nothing. + */ + public void testRollbackNoTransaction() { + execute(node(), "ROLLBACK"); + } + + /** + * Test that ROLLBACK rolls back a transaction. + */ + public void testRollback() { + execute(node(), "BEGIN TRANSACTION"); + + assertTxPresent(); + + Transaction tx = tx(); + + assertTxState(tx, TransactionState.ACTIVE); + + execute(node(), "ROLLBACK TRANSACTION"); + + assertTxState(tx, TransactionState.ROLLED_BACK); + + assertSqlTxNotPresent(); + } + + /** + * Test that attempting to perform various SQL operations within non SQL transaction yields an exception. + */ + public void testSqlOperationsWithinNonSqlTransaction() { + assertSqlOperationWithinNonSqlTransactionThrows("COMMIT"); + + assertSqlOperationWithinNonSqlTransactionThrows("ROLLBACK"); + + assertSqlOperationWithinNonSqlTransactionThrows("SELECT * from ints"); + + assertSqlOperationWithinNonSqlTransactionThrows("DELETE from ints"); + + assertSqlOperationWithinNonSqlTransactionThrows("INSERT INTO ints(k, v) values(10, 15)"); + + assertSqlOperationWithinNonSqlTransactionThrows("MERGE INTO ints(k, v) values(10, 15)"); + + assertSqlOperationWithinNonSqlTransactionThrows("UPDATE ints SET v = 100 WHERE k = 5"); + + assertSqlOperationWithinNonSqlTransactionThrows("create index idx on ints(v)"); + + assertSqlOperationWithinNonSqlTransactionThrows("CREATE TABLE T(k int primary key, v int)"); + } + + /** + * Check that trying to run given SQL statement both locally and in distributed mode yields an exception + * if transaction already has been marked as being of SQL type. + * @param sql SQL statement. + */ + private void assertSqlOperationWithinNonSqlTransactionThrows(final String sql) { + try (Transaction ignored = node().transactions().txStart()) { + node().cache("ints").put(1, 1); + + assertSqlException(new RunnableX() { + @Override public void run() throws Exception { + execute(node(), sql); + } + }, IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH); + } + + try (Transaction ignored = node().transactions().txStart()) { + node().cache("ints").put(1, 1); + + assertSqlException(new RunnableX() { + @Override public void run() throws Exception { + node().cache("ints").query(new SqlFieldsQuery(sql).setLocal(true)).getAll(); + } + }, IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH); + } + } + + /** + * Test that attempting to perform a cache API operation from within an SQL transaction fails. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + private void checkCacheOperationThrows(final String opName, final Object... args) { + execute(node(), "BEGIN"); + + try { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + try { + // We need to detect types based on arguments due to multiple overloads. + Class[] types; + + if (F.isEmpty(args)) + types = (Class[]) X.EMPTY_OBJECT_ARRAY; + else { + types = new Class[args.length]; + + for (int i = 0; i < args.length; i++) + types[i] = argTypeForObject(args[i]); + } + + Object res = U.invoke(GatewayProtectedCacheProxy.class, node().cache("ints"), + opName, types, args); + + if (opName.endsWith("Async")) + ((IgniteFuture)res).get(); + } + catch (IgniteCheckedException e) { + if (e.getCause() != null) { + try { + if (e.getCause().getCause() != null) + throw (Exception)e.getCause().getCause(); + else + fail(); + } + catch (IgniteException e1) { + // Some public API methods don't have IgniteCheckedException on their signature + // and thus may wrap it into an IgniteException. + if (e1.getCause() != null) + throw (Exception)e1.getCause(); + else + fail(); + } + } + else + fail(); + } + + return null; + } + }, IgniteCheckedException.class, + "SQL queries and cache operations may not be used in the same transaction."); + } + finally { + try { + execute(node(), "ROLLBACK"); + } + catch (Throwable e) { + // No-op. + } + } + } + + /** + * + */ + private static Class argTypeForObject(Object arg) { + if (arg instanceof Set) + return Set.class; + else if (arg instanceof Map) + return Map.class; + else if (arg.getClass().getName().startsWith("java.lang.")) + return Object.class; + else if (arg instanceof CacheEntryProcessor) + return CacheEntryProcessor.class; + else if (arg instanceof EntryProcessor) + return EntryProcessor.class; + else + return arg.getClass(); + } + + /** + * Test that attempting to perform a cache PUT operation from within an SQL transaction fails. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCacheOperationsFromSqlTransaction() { + checkCacheOperationThrows("get", 1); + + checkCacheOperationThrows("getAsync", 1); + + checkCacheOperationThrows("getEntry", 1); + + checkCacheOperationThrows("getEntryAsync", 1); + + checkCacheOperationThrows("getAndPut", 1, 1); + + checkCacheOperationThrows("getAndPutAsync", 1, 1); + + checkCacheOperationThrows("getAndPutIfAbsent", 1, 1); + + checkCacheOperationThrows("getAndPutIfAbsentAsync", 1, 1); + + checkCacheOperationThrows("getAndReplace", 1, 1); + + checkCacheOperationThrows("getAndReplaceAsync", 1, 1); + + checkCacheOperationThrows("getAndRemove", 1); + + checkCacheOperationThrows("getAndRemoveAsync", 1); + + checkCacheOperationThrows("containsKey", 1); + + checkCacheOperationThrows("containsKeyAsync", 1); + + checkCacheOperationThrows("put", 1, 1); + + checkCacheOperationThrows("putAsync", 1, 1); + + checkCacheOperationThrows("putIfAbsent", 1, 1); + + checkCacheOperationThrows("putIfAbsentAsync", 1, 1); + + checkCacheOperationThrows("remove", 1); + + checkCacheOperationThrows("removeAsync", 1); + + checkCacheOperationThrows("remove", 1, 1); + + checkCacheOperationThrows("removeAsync", 1, 1); + + checkCacheOperationThrows("replace", 1, 1); + + checkCacheOperationThrows("replaceAsync", 1, 1); + + checkCacheOperationThrows("replace", 1, 1, 1); + + checkCacheOperationThrows("replaceAsync", 1, 1, 1); + + checkCacheOperationThrows("getAll", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("containsKeys", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("getEntries", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("putAll", Collections.singletonMap(1, 1)); + + checkCacheOperationThrows("removeAll", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("getAllAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("containsKeysAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("getEntriesAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("putAllAsync", Collections.singletonMap(1, 1)); + + checkCacheOperationThrows("removeAllAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("invoke", 1, ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invoke", 1, CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAsync", 1, ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAsync", 1, CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAll", Collections.singletonMap(1, CACHE_ENTRY_PROC), X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAll", Collections.singleton(1), CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAll", Collections.singleton(1), ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAllAsync", Collections.singletonMap(1, CACHE_ENTRY_PROC), + X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAllAsync", Collections.singleton(1), CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAllAsync", Collections.singleton(1), ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + } + + /** */ + private final static EntryProcessor ENTRY_PROC = + new EntryProcessor() { + @Override public Object process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + return null; + } + }; + + /** */ + private final static CacheEntryProcessor CACHE_ENTRY_PROC = + new CacheEntryProcessor() { + @Override public Object process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + return null; + } + }; + + /** + * @return Node. + */ + private IgniteEx node() { + return grid(0); + } + + /** + * @return Currently open transaction. + */ + private Transaction tx() { + return node().transactions().tx(); + } + + /** + * Check that there's an open transaction with SQL flag. + */ + private void assertTxPresent() { + assertNotNull(tx()); + } + + /** {@inheritDoc} */ + @Override protected List> execute(Ignite node, String sql) { + return node.cache("ints").query(new SqlFieldsQuery(sql).setSchema(QueryUtils.DFLT_SCHEMA)).getAll(); + } + + /** + * Check that there's no open transaction. + */ + private void assertSqlTxNotPresent() { + assertNull(tx()); + } + + /** + * Check transaction state. + */ + private static void assertTxState(Transaction tx, TransactionState state) { + assertEquals(state, tx.state()); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlCoordinatorFailoverTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlCoordinatorFailoverTest.java new file mode 100644 index 0000000000000..c449ee20e390b --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlCoordinatorFailoverTest.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.internal.processors.cache.mvcc; + +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SCAN; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.DML; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Mvcc SQL API coordinator failover test. + */ +@SuppressWarnings("unchecked") +public abstract class CacheMvccAbstractSqlCoordinatorFailoverTest extends CacheMvccAbstractBasicCoordinatorFailoverTest { + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_Server_Backups0_CoordinatorFails() throws Exception { + accountsTxReadAll(2, 1, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_SingleNode_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + accountsTxReadAll(1, 0, 0, 1, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups0_RestartCoordinator_ScanDml() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD , 2, 1, 0, 64, + new InitIndexing(Integer.class, Integer.class), SCAN, DML); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_SingleNode_RestartCoordinator_ScanDml_Persistence() throws Exception { + persistence = true; + + putAllGetAll(RestartMode.RESTART_CRD , 1, 0, 0, 1, + new InitIndexing(Integer.class, Integer.class), SCAN, DML); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups0_RestartCoordinator_SqlDml() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD, 2, 1, 0, DFLT_PARTITION_COUNT, + new InitIndexing(Integer.class, Integer.class), SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_SingleNode_RestartCoordinator_SqlDml_Persistence() throws Exception { + persistence = true; + + putAllGetAll(RestartMode.RESTART_CRD, 1, 0, 0, 1, + new InitIndexing(Integer.class, Integer.class), SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_ClientServer_Backups0_Sql_Persistence() throws Exception { + persistence = true; + + updateNObjectsTest(5, 2, 0, 0, 64, DFLT_TEST_TIME, + new InitIndexing(Integer.class, Integer.class), SQL, DML, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_SingleNode_Sql_Persistence() throws Exception { + updateNObjectsTest(3, 1, 0, 0, 1, DFLT_TEST_TIME, + new InitIndexing(Integer.class, Integer.class), SQL, DML, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testCoordinatorFailureSimplePessimisticTxSql() throws Exception { + coordinatorFailureSimple(PESSIMISTIC, REPEATABLE_READ, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testTxInProgressCoordinatorChangeSimple_Readonly() throws Exception { + txInProgressCoordinatorChangeSimple(PESSIMISTIC, REPEATABLE_READ, + new InitIndexing(Integer.class, Integer.class), SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testReadInProgressCoordinatorFailsSimple_FromClient() throws Exception { + readInProgressCoordinatorFailsSimple(true, new InitIndexing(Integer.class, Integer.class), SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testCoordinatorChangeActiveQueryClientFails_Simple() throws Exception { + checkCoordinatorChangeActiveQueryClientFails_Simple(new InitIndexing(Integer.class, Integer.class), SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testCoordinatorChangeActiveQueryClientFails_SimpleScan() throws Exception { + checkCoordinatorChangeActiveQueryClientFails_Simple(new InitIndexing(Integer.class, Integer.class), SCAN, DML); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java new file mode 100644 index 0000000000000..71e004bfbd8f5 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java @@ -0,0 +1,808 @@ +/* + * 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.mvcc; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +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.affinity.Affinity; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryEnlistResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.transactions.Transaction; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.DML; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; +import static org.junit.Assert.assertArrayEquals; + +/** + * Backups tests. + */ +@SuppressWarnings("unchecked") +public abstract class CacheMvccBackupsAbstractTest extends CacheMvccAbstractTest { + + /** Test timeout. */ + private final long txLongTimeout = getTestTimeout() / 4; + + /** + * Tests backup consistency. + * + * @throws Exception If fails. + */ + public void testBackupsCoherenceSimple() throws Exception { + disableScheduledVacuum = true; + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + final int KEYS_CNT = 5_000; + assert KEYS_CNT % 2 == 0; + + startGrids(3); + + Ignite node0 = grid(0); + Ignite node1 = grid(1); + Ignite node2 = grid(2); + + client = true; + + Ignite client = startGrid(); + + awaitPartitionMapExchange(); + + IgniteCache clientCache = client.cache(DEFAULT_CACHE_NAME); + IgniteCache cache0 = node0.cache(DEFAULT_CACHE_NAME); + IgniteCache cache1 = node1.cache(DEFAULT_CACHE_NAME); + IgniteCache cache2 = node2.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + for (int i = 0; i < KEYS_CNT / 2; i += 2) { + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (" + + i + ',' + i * 2 + "), (" + (i + 1) + ',' + (i + 1) * 2 + ')'); + + clientCache.query(qry).getAll(); + } + + tx.commit(); + } + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + for (int i = 0; i < 10; i++) { + SqlFieldsQuery qry = new SqlFieldsQuery("DELETE from Integer WHERE _key = " + i); + + clientCache.query(qry).getAll(); + } + + for (int i = 10; i < KEYS_CNT + 1; i++) { + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE Integer SET _val=" + i * 10 + " WHERE _key = " + i); + + clientCache.query(qry).getAll(); + } + + tx.commit(); + } + + Map> vers0 = allVersions(cache0); + + List res0 = getAll(cache0, "Integer"); + + stopGrid(0); + + awaitPartitionMapExchange(); + + Map> vers1 = allVersions(cache1); + + assertVersionsEquals(vers0, vers1); + + List res1 = getAll(cache1, "Integer"); + + assertEqualsCollections(res0, res1); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + for (int i = 10; i < 20; i++) { + SqlFieldsQuery qry = new SqlFieldsQuery("DELETE from Integer WHERE _key = " + i); + + clientCache.query(qry).getAll(); + } + + for (int i = 20; i < KEYS_CNT + 1; i++) { + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE Integer SET _val=" + i * 100 + " WHERE _key = " + i); + + clientCache.query(qry).getAll(); + } + + tx.commit(); + } + + vers1 = allVersions(cache1); + + res1 = getAll(cache2, "Integer"); + + stopGrid(1); + + awaitPartitionMapExchange(); + + Map> vers2 = allVersions(cache2); + + assertVersionsEquals(vers1, vers2); + + List res2 = getAll(cache2, "Integer"); + + assertEqualsCollections(res1, res2); + } + + /** + * Checks cache backups consistency with large queries. + * + * @throws Exception If failed. + */ + public void testBackupsCoherenceWithLargeOperations() throws Exception { + disableScheduledVacuum = true; + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 1, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + final int KEYS_CNT = 50_000; + assert KEYS_CNT % 2 == 0; + + startGrids(2); + + Ignite node1 = grid(0); + Ignite node2 = grid(1); + + client = true; + + Ignite client = startGrid(); + + awaitPartitionMapExchange(); + + IgniteCache clientCache = client.cache(DEFAULT_CACHE_NAME); + IgniteCache cache1 = node1.cache(DEFAULT_CACHE_NAME); + IgniteCache cache2 = node2.cache(DEFAULT_CACHE_NAME); + + StringBuilder insert = new StringBuilder("INSERT INTO Integer (_key, _val) values "); + + boolean first = true; + + for (int key = 0; key < KEYS_CNT; key++) { + if (!first) + insert.append(','); + else + first = false; + + insert.append('(').append(key).append(',').append(key * 10).append(')'); + } + + String qryStr = insert.toString(); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + SqlFieldsQuery qry = new SqlFieldsQuery(qryStr); + + clientCache.query(qry).getAll(); + + tx.commit(); + } + + qryStr = "SELECT * FROM Integer WHERE _key >= " + KEYS_CNT / 2 + " FOR UPDATE"; + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + SqlFieldsQuery qry = new SqlFieldsQuery(qryStr); + + clientCache.query(qry).getAll(); + + tx.commit(); + } + + + qryStr = "DELETE FROM Integer WHERE _key >= " + KEYS_CNT / 2; + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + SqlFieldsQuery qry = new SqlFieldsQuery(qryStr); + + clientCache.query(qry).getAll(); + + tx.commit(); + } + + Map> cache1Vers = allVersions(cache1); + + List res1 = getAll(cache1, "Integer"); + + stopGrid(0); + + awaitPartitionMapExchange(); + + Map> cache2Vers = allVersions(cache2); + + assertVersionsEquals(cache1Vers, cache2Vers); + + List res2 = getAll(cache2, "Integer"); + + assertEqualsCollections(res1, res2); + } + + /** + * Checks cache backups consistency with in-flight batches overflow. + * + * @throws Exception If failed. + */ + public void testBackupsCoherenceWithInFlightBatchesOverflow() throws Exception { + testSpi = true; + + disableScheduledVacuum = true; + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 1, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + final int KEYS_CNT = 30_000; + assert KEYS_CNT % 2 == 0; + + startGrids(2); + + Ignite node1 = grid(0); + Ignite node2 = grid(1); + + client = true; + + Ignite client = startGrid(); + + awaitPartitionMapExchange(); + + IgniteCache clientCache = client.cache(DEFAULT_CACHE_NAME); + IgniteCache cache1 = node1.cache(DEFAULT_CACHE_NAME); + IgniteCache cache2 = node2.cache(DEFAULT_CACHE_NAME); + + AtomicInteger keyGen = new AtomicInteger(); + Affinity affinity = affinity(clientCache); + + ClusterNode cNode1 = ((IgniteEx)node1).localNode(); + ClusterNode cNode2 = ((IgniteEx)node2).localNode(); + + StringBuilder insert = new StringBuilder("INSERT INTO Integer (_key, _val) values "); + + for (int i = 0; i < KEYS_CNT; i++) { + if (i > 0) + insert.append(','); + + // To make big batches in near results future. + Integer key = i < KEYS_CNT / 2 ? keyForNode(affinity, keyGen, cNode1) : keyForNode(affinity, keyGen, cNode2); + + assert key != null; + + insert.append('(').append(key).append(',').append(key * 10).append(')'); + } + + String qryStr = insert.toString(); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + SqlFieldsQuery qry = new SqlFieldsQuery(qryStr); + + clientCache.query(qry).getAll(); + + tx.commit(); + } + + // Add a delay to simulate batches overflow. + TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(node1); + TestRecordingCommunicationSpi spi2 = TestRecordingCommunicationSpi.spi(node2); + + spi1.closure(new IgniteBiInClosure() { + @Override public void apply(ClusterNode node, Message msg) { + if (msg instanceof GridDhtTxQueryEnlistResponse) + doSleep(100); + } + }); + + spi2.closure(new IgniteBiInClosure() { + @Override public void apply(ClusterNode node, Message msg) { + if (msg instanceof GridDhtTxQueryEnlistResponse) + doSleep(100); + } + }); + + qryStr = "DELETE FROM Integer WHERE _key >= " + 10; + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(txLongTimeout); + + SqlFieldsQuery qry = new SqlFieldsQuery(qryStr); + + clientCache.query(qry).getAll(); + + tx.commit(); + } + + Map> cache1Vers = allVersions(cache1); + + List res1 = getAll(cache1, "Integer"); + + stopGrid(0); + + awaitPartitionMapExchange(); + + Map> cache2Vers = allVersions(cache2); + + assertVersionsEquals(cache1Vers, cache2Vers); + + List res2 = getAll(cache2, "Integer"); + + assertEqualsCollections(res1, res2); + } + + /** + * Tests concurrent updates backups coherence. + * + * @throws Exception If failed. + */ + public void testBackupsCoherenceWithConcurrentUpdates2ServersNoClients() throws Exception { + checkBackupsCoherenceWithConcurrentUpdates(2, 0); + } + + /** + * Tests concurrent updates backups coherence. + * + * @throws Exception If failed. + */ + public void testBackupsCoherenceWithConcurrentUpdates4ServersNoClients() throws Exception { + checkBackupsCoherenceWithConcurrentUpdates(4, 0); + } + + /** + * Tests concurrent updates backups coherence. + * + * @throws Exception If failed. + */ + public void testBackupsCoherenceWithConcurrentUpdates3Servers1Client() throws Exception { + checkBackupsCoherenceWithConcurrentUpdates(3, 1); + } + + /** + * Tests concurrent updates backups coherence. + * + * @throws Exception If failed. + */ + public void testBackupsCoherenceWithConcurrentUpdates5Servers2Clients() throws Exception { + checkBackupsCoherenceWithConcurrentUpdates(5, 2); + } + + /** + * Tests concurrent updates backups coherence. + * + * @throws Exception If failed. + */ + private void checkBackupsCoherenceWithConcurrentUpdates(int srvs, int clients) throws Exception { + assert srvs > 1; + + disableScheduledVacuum = true; + + accountsTxReadAll(srvs, clients, srvs - 1, DFLT_PARTITION_COUNT, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML, 5_000, null); + + for (int i = 0; i < srvs - 1; i++) { + Ignite node1 = grid(i); + + IgniteCache cache1 = node1.cache(DEFAULT_CACHE_NAME); + + Map> vers1 = allVersions(cache1); + + List res1 = getAll(cache1, "MvccTestAccount"); + + stopGrid(i); + + awaitPartitionMapExchange(); + + Ignite node2 = grid(i + 1); + + IgniteCache cache2 = node2.cache(DEFAULT_CACHE_NAME); + + Map> vers2 = allVersions(cache2); + + assertVersionsEquals(vers1, vers2); + + List res2 = getAll(cache2, "MvccTestAccount"); + + assertEqualsCollections(res1, res2); + } + } + + /** + * @throws Exception If failed. + */ + public void testNoForceKeyRequestDelayedRebalanceNoVacuum() throws Exception { + disableScheduledVacuum = true; + + doTestRebalanceNodeAdd(true); + } + + /** + * @throws Exception If failed. + */ + public void testNoForceKeyRequestDelayedRebalance() throws Exception { + doTestRebalanceNodeAdd(true); + } + + /** + * @throws Exception If failed. + */ + public void testNoForceKeyRequestNoVacuum() throws Exception { + disableScheduledVacuum = true; + + doTestRebalanceNodeAdd(false); + } + + /** + * @throws Exception If failed. + */ + public void testNoForceKeyRequest() throws Exception { + doTestRebalanceNodeAdd(false); + } + + /** + * @throws Exception If failed. + */ + private void doTestRebalanceNodeAdd(boolean delayRebalance) throws Exception { + testSpi = true; + + final Ignite node1 = startGrid(0); + + final IgniteCache cache = node1.createCache( + cacheConfiguration(cacheMode(), FULL_SYNC, 1, 16) + .setIndexedTypes(Integer.class, Integer.class)); + + try (Transaction tx = node1.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values " + + "(1,1),(2,2),(3,3),(4,4),(5,5)"); + + cache.query(qry).getAll(); + + tx.commit(); + } + + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(node1); + + // Check for a force key request. + spi.closure(new IgniteBiInClosure() { + @Override public void apply(ClusterNode node, Message msg) { + if (delayRebalance && msg instanceof GridDhtPartitionSupplyMessage) + doSleep(500); + + if (msg instanceof GridDhtForceKeysResponse) + fail("Force key request"); + } + }); + + final Ignite node2 = startGrid(1); + + TestRecordingCommunicationSpi.spi(node2).closure( + new IgniteBiInClosure() { + @Override public void apply(ClusterNode node, Message msg) { + if (msg instanceof GridDhtForceKeysRequest) + fail("Force key request"); + } + } + ); + + IgniteCache cache2 = node2.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = node2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + SqlFieldsQuery qry = new SqlFieldsQuery("DELETE FROM Integer WHERE _key IN " + + "(1,2,3,4,5)"); + + cache2.query(qry).getAll(); + + tx.commit(); + } + + awaitPartitionMapExchange(); + + doSleep(2000); + + stopGrid(1); + + try (Transaction tx = node1.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values " + + "(1,1),(2,2),(3,3),(4,4),(5,5)"); + + cache.query(qry).getAll(); + + tx.commit(); + } + + doSleep(1000); + } + + /** + * @throws Exception If failed. + */ + public void testRebalanceNodeLeaveClient() throws Exception { + doTestRebalanceNodeLeave(true); + } + + /** + * @throws Exception If failed. + */ + public void testRebalanceNodeLeaveServer() throws Exception { + doTestRebalanceNodeLeave(false); + } + + /** + * @throws Exception If failed. + */ + public void doTestRebalanceNodeLeave(boolean startClient) throws Exception { + testSpi = true; + disableScheduledVacuum = true; + + startGridsMultiThreaded(4); + + client = true; + + final Ignite node = startClient ? startGrid(4) : grid(0); + + final IgniteCache cache = node.createCache( + cacheConfiguration(cacheMode(), FULL_SYNC, 2, 16) + .setIndexedTypes(Integer.class, Integer.class)); + + List keys = new ArrayList<>(); + + for (int i = 0; i < 4; i++) + keys.addAll(primaryKeys(grid(i).cache(DEFAULT_CACHE_NAME), 2)); + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + StringBuilder sb = new StringBuilder("INSERT INTO Integer (_key, _val) values "); + + for (int i = 0; i < keys.size(); i++) { + if (i > 0) + sb.append(", "); + + sb.append("(" + keys.get(i) + ", " + keys.get(i) + ")"); + } + + SqlFieldsQuery qry = new SqlFieldsQuery(sb.toString()); + + cache.query(qry).getAll(); + + tx.commit(); + } + + stopGrid(3); + + awaitPartitionMapExchange(); + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE Integer SET _val = 10*_key"); + + cache.query(qry).getAll(); + + tx.commit(); + } + + awaitPartitionMapExchange(); + + for (Integer key : keys) { + List vers = null; + + for (int i = 0; i < 3; i++) { + ClusterNode n = grid(i).cluster().localNode(); + + if (node.affinity(DEFAULT_CACHE_NAME).isPrimaryOrBackup(n, key)) { + + List vers0 = allKeyVersions(grid(i).cache(DEFAULT_CACHE_NAME), key); + + if (vers != null) + assertKeyVersionsEquals(vers, vers0); + + vers = vers0; + } + } + } + } + + /** + * Retrieves all versions of all keys from cache. + * + * @param cache Cache. + * @return {@link Map} of keys to its versions. + * @throws IgniteCheckedException If failed. + */ + private Map> allVersions(IgniteCache cache) throws IgniteCheckedException { + IgniteCacheProxy cache0 = (IgniteCacheProxy)cache; + GridCacheContext cctx = cache0.context(); + + assert cctx.mvccEnabled(); + + Map> vers = new HashMap<>(); + + for (Object e : cache) { + IgniteBiTuple entry = (IgniteBiTuple)e; + + KeyCacheObject key = cctx.toCacheKeyObject(entry.getKey()); + + GridCursor cur = cctx.offheap().mvccAllVersionsCursor(cctx, key, null); + + List rows = new ArrayList<>(); + + while (cur.next()) { + CacheDataRow row = cur.get(); + + rows.add(row); + } + + vers.put(key, rows); + } + + return vers; + } + + /** + * @param cache Cache. + * @param key Key. + * @return Collection of versioned rows. + * @throws IgniteCheckedException if failed. + */ + private List allKeyVersions(IgniteCache cache, Object key) throws IgniteCheckedException { + IgniteCacheProxy cache0 = (IgniteCacheProxy)cache; + GridCacheContext cctx = cache0.context(); + + KeyCacheObject key0 = cctx.toCacheKeyObject(key); + + GridCursor cur = cctx.offheap().mvccAllVersionsCursor(cctx, key0, null); + + List rows = new ArrayList<>(); + + while (cur.next()) { + CacheDataRow row = cur.get(); + + rows.add(row); + } + + return rows; + } + + /** + * Checks stored versions equality. + * + * @param left Keys versions to compare. + * @param right Keys versions to compare. + * @throws IgniteCheckedException If failed. + */ + private void assertVersionsEquals(Map> left, + Map> right) throws IgniteCheckedException { + assertNotNull(left); + assertNotNull(right); + + assertTrue(!left.isEmpty()); + assertTrue(!right.isEmpty()); + + assertEqualsCollections(left.keySet(), right.keySet()); + + for (KeyCacheObject key : right.keySet()) { + List leftRows = left.get(key); + List rightRows = right.get(key); + + assertKeyVersionsEquals(leftRows, rightRows); + } + } + + /** + * + * @param leftRows Left rows. + * @param rightRows Right rows. + * @throws IgniteCheckedException If failed. + */ + private void assertKeyVersionsEquals(List leftRows, List rightRows) + throws IgniteCheckedException { + + assertNotNull(leftRows); + assertNotNull(rightRows); + + assertEquals("leftRows=" + leftRows + ", rightRows=" + rightRows, leftRows.size(), rightRows.size()); + + for (int i = 0; i < leftRows.size(); i++) { + CacheDataRow leftRow = leftRows.get(i); + CacheDataRow rightRow = rightRows.get(i); + + assertNotNull(leftRow); + assertNotNull(rightRow); + + assertTrue(leftRow instanceof MvccDataRow); + assertTrue(rightRow instanceof MvccDataRow); + + leftRow.key().valueBytes(null); + + assertEquals(leftRow.expireTime(), rightRow.expireTime()); + assertEquals(leftRow.partition(), rightRow.partition()); + assertArrayEquals(leftRow.value().valueBytes(null), rightRow.value().valueBytes(null)); + assertEquals(leftRow.version(), rightRow.version()); + assertEquals(leftRow.cacheId(), rightRow.cacheId()); + assertEquals(leftRow.hash(), rightRow.hash()); + assertEquals(leftRow.key(), rightRow.key()); + assertTrue(MvccUtils.compare(leftRow, rightRow.mvccVersion()) == 0); + assertTrue(MvccUtils.compareNewVersion(leftRow, rightRow.newMvccVersion()) == 0); + assertEquals(leftRow.newMvccCoordinatorVersion(), rightRow.newMvccCoordinatorVersion()); + assertEquals(leftRow.newMvccCounter(), rightRow.newMvccCounter()); + assertEquals(leftRow.newMvccOperationCounter(), rightRow.newMvccOperationCounter()); + } + } + + /** + * Retrieves all table rows from local node. + * @param cache Cache. + * @param tblName Table name. + * @return All table rows. + */ + private List getAll(IgniteCache cache, String tblName) { + List res = cache.query(new SqlFieldsQuery("SELECT * FROM " + tblName)).getAll(); + + Collections.sort(res, new Comparator() { + @Override public int compare(Object o1, Object o2) { + List l1 = (List)o1; + List l2 = (List)o2; + + int res = ((Comparable)l1.get(0)).compareTo((Comparable)l2.get(0)); + + if (res == 0 && l1.size() > 1) + return ((Comparable)l1.get(1)).compareTo((Comparable)l2.get(1)); + else + return res; + + } + }); + + return res; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBulkLoadTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBulkLoadTest.java new file mode 100644 index 0000000000000..d620e840b1f84 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBulkLoadTest.java @@ -0,0 +1,123 @@ +/* + * 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.mvcc; + +import java.io.File; +import java.io.Serializable; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; + +/** + * + */ +public class CacheMvccBulkLoadTest extends CacheMvccAbstractTest { + /** */ + private IgniteCache sqlNexus; + + /** */ + private Statement stmt; + + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + Ignite ignite = startGrid(0); + sqlNexus = ignite.getOrCreateCache(new CacheConfiguration<>("sqlNexus").setSqlSchema("PUBLIC")); + sqlNexus.query(q("" + + "create table person(" + + " id int not null primary key," + + " name varchar not null" + + ") with \"atomicity=transactional\"" + )); + stmt = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1").createStatement(); + } + + /** + * @throws Exception If failed. + */ + public void testCopyStoresData() throws Exception { + String csvFilePath = new File(getClass().getResource("mvcc_person.csv").toURI()).getAbsolutePath(); + stmt.executeUpdate("copy from '" + csvFilePath + "' into person (id, name) format csv"); + + List> rows = sqlNexus.query(q("select * from person")).getAll(); + + List> exp = asList( + asList(1, "John"), + asList(2, "Jack") + ); + assertEquals(exp, rows); + } + + /** + * @throws Exception If failed. + */ + public void testCopyDoesNotOverwrite() throws Exception { + sqlNexus.query(q("insert into person values(1, 'Old')")); + String csvFilePath = new File(getClass().getResource("mvcc_person.csv").toURI()).getAbsolutePath(); + stmt.executeUpdate("copy from '" + csvFilePath + "' into person (id, name) format csv"); + + List> rows = sqlNexus.query(q("select * from person")).getAll(); + + List> exp = asList( + asList(1, "Old"), + asList(2, "Jack") + ); + assertEquals(exp, rows); + } + + /** + * @throws Exception If failed. + */ + public void testCopyLeavesPartialResultsInCaseOfFailure() throws Exception { + String csvFilePath = new File(getClass().getResource("mvcc_person_broken.csv").toURI()).getAbsolutePath(); + try { + stmt.executeUpdate("copy from '" + csvFilePath + "' into person (id, name) format csv"); + fail(); + } + catch (SQLException ignored) { + // assert exception is thrown + } + + List> rows = sqlNexus.query(q("select * from person")).getAll(); + + List> exp = singletonList( + asList(1, "John") + ); + assertEquals(exp, rows); + } + + /** */ + private static SqlFieldsQuery q(String sql) { + return new SqlFieldsQuery(sql); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccDmlSimpleTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccDmlSimpleTest.java new file mode 100644 index 0000000000000..bb5e7536bbe42 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccDmlSimpleTest.java @@ -0,0 +1,200 @@ +/* + * 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.mvcc; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import javax.cache.CacheException; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.query.IgniteSQLException; + +import static java.util.Arrays.asList; + +/** + * + */ +public class CacheMvccDmlSimpleTest extends CacheMvccAbstractTest { + /** */ + private IgniteCache cache; + + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + cache = startGrid(0).getOrCreateCache( + new CacheConfiguration<>("test") + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setSqlSchema("PUBLIC") + .setIndexedTypes(Integer.class, Integer.class) + ); + } + + /** + * @throws Exception if failed. + */ + public void testInsert() throws Exception { + int cnt = update("insert into Integer(_key, _val) values(1, 1),(2, 2)"); + + assertEquals(2, cnt); + + assertEquals(asSet(asList(1, 1), asList(2, 2)), query("select * from Integer")); + + try { + update("insert into Integer(_key, _val) values(3, 3),(1, 1)"); + } catch (CacheException e) { + assertTrue(e.getCause() instanceof IgniteSQLException); + assertEquals(IgniteQueryErrorCode.DUPLICATE_KEY, ((IgniteSQLException)e.getCause()).statusCode()); + } + + assertEquals(asSet(asList(1, 1), asList(2, 2)), query("select * from Integer")); + } + + /** + * @throws Exception if failed. + */ + public void testMerge() throws Exception { + { + int cnt = update("merge into Integer(_key, _val) values(1, 1),(2, 2)"); + + assertEquals(2, cnt); + assertEquals(asSet(asList(1, 1), asList(2, 2)), query("select * from Integer")); + } + + { + int cnt = update("merge into Integer(_key, _val) values(3, 3),(1, 1)"); + + assertEquals(2, cnt); + assertEquals(asSet(asList(1, 1), asList(2, 2), asList(3, 3)), query("select * from Integer")); + } + } + + /** + * @throws Exception if failed. + */ + public void testUpdate() throws Exception { + { + int cnt = update("update Integer set _val = 42 where _key = 42"); + + assertEquals(0, cnt); + assertTrue(query("select * from Integer").isEmpty()); + } + + update("insert into Integer(_key, _val) values(1, 1),(2, 2)"); + + { + int cnt = update("update Integer set _val = 42 where _key = 42"); + + assertEquals(0, cnt); + assertEquals(asSet(asList(1, 1), asList(2, 2)), query("select * from Integer")); + } + + { + int cnt = update("update Integer set _val = 42 where _key >= 42"); + + assertEquals(0, cnt); + assertEquals(asSet(asList(1, 1), asList(2, 2)), query("select * from Integer")); + } + + { + int cnt = update("update Integer set _val = 11 where _key = 1"); + + assertEquals(1, cnt); + assertEquals(asSet(asList(1, 11), asList(2, 2)), query("select * from Integer")); + } + + { + int cnt = update("update Integer set _val = 12 where _key <= 2"); + + assertEquals(asSet(asList(1, 12), asList(2, 12)), query("select * from Integer")); + assertEquals(2, cnt); + } + } + + /** + * @throws Exception if failed. + */ + public void testDelete() throws Exception { + { + int cnt = update("delete from Integer where _key = 42"); + + assertEquals(0, cnt); + } + + update("insert into Integer(_key, _val) values(1, 1),(2, 2)"); + + { + int cnt = update("delete from Integer where _key = 42"); + + assertEquals(0, cnt); + assertEquals(asSet(asList(1, 1), asList(2, 2)), query("select * from Integer")); + } + + { + int cnt = update("delete from Integer where _key >= 42"); + + assertEquals(0, cnt); + assertEquals(asSet(asList(1, 1), asList(2, 2)), query("select * from Integer")); + } + + { + int cnt = update("delete from Integer where _key = 1"); + + assertEquals(1, cnt); + assertEquals(asSet(asList(2, 2)), query("select * from Integer")); + } + + { + int cnt = update("delete from Integer where _key <= 2"); + + assertTrue(query("select * from Integer").isEmpty()); + assertEquals(1, cnt); + } + } + + /** + * @param q Query. + * @return Row set. + */ + private Set> query(String q) { + return new HashSet<>(cache.query(new SqlFieldsQuery(q)).getAll()); + } + + /** + * @param q Query. + * @return Updated rows count. + */ + private int update(String q) { + return Integer.parseInt(cache.query(new SqlFieldsQuery(q)).getAll().get(0).get(0).toString()); + } + + /** */ + private Set> asSet(List... ls) { + return new HashSet<>(asList(ls)); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentJdbcTransactionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentJdbcTransactionTest.java new file mode 100644 index 0000000000000..235d87fcaccdb --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccIteratorWithConcurrentJdbcTransactionTest.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.cache.mvcc; + +/** + * + */ +public class CacheMvccIteratorWithConcurrentJdbcTransactionTest extends CacheMvccIteratorWithConcurrentTransactionTest { + /** {@inheritDoc} */ + @Override boolean jdbcTx() { + return true; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest.java new file mode 100644 index 0000000000000..97c062f2aa796 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest.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.internal.processors.cache.mvcc; + +/** + * + */ +public class CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest extends + CacheMvccLocalEntriesWithConcurrentTransactionTest { + /** {@inheritDoc} */ + @Override boolean jdbcTx() { + return true; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedBackupsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedBackupsTest.java new file mode 100644 index 0000000000000..71d832c1e2b14 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedBackupsTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** */ +public class CacheMvccPartitionedBackupsTest extends CacheMvccBackupsAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return PARTITIONED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSelectForUpdateQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSelectForUpdateQueryTest.java new file mode 100644 index 0000000000000..12209abd4f389 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSelectForUpdateQueryTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** */ +public class CacheMvccPartitionedSelectForUpdateQueryTest extends CacheMvccSelectForUpdateQueryAbstractTest { + /** {@inheritDoc} */ + public CacheMode cacheMode() { + return PARTITIONED; + } + + /** + * + */ + public void testSelectForUpdateDistributedSegmented() throws Exception { + doTestSelectForUpdateDistributed("PersonSeg", false); + } + + /** + * + */ + public void testSelectForUpdateLocalSegmented() throws Exception { + doTestSelectForUpdateLocal("PersonSeg", false); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java new file mode 100644 index 0000000000000..1362b4a6c7d1e --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlCoordinatorFailoverTest.java @@ -0,0 +1,143 @@ +/* + * 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.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SCAN; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.DML; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * SQL Mvcc coordinator failover test for partitioned caches. + */ +public class CacheMvccPartitionedSqlCoordinatorFailoverTest extends CacheMvccAbstractSqlCoordinatorFailoverTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_ClientServer_Backups2_CoordinatorFails() throws Exception { + accountsTxReadAll(4, 2, 2, DFLT_PARTITION_COUNT, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_Server_Backups1_CoordinatorFails_Persistence() throws Exception { + persistence = true; + + accountsTxReadAll(2, 0, 1, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML, DFLT_TEST_TIME, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups3_RestartCoordinator_ScanDml() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD , 5, 2, 3, DFLT_PARTITION_COUNT, + new InitIndexing(Integer.class, Integer.class), SCAN, DML); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_RestartCoordinator_ScanDml_Persistence() throws Exception { + persistence = true; + + putAllGetAll(RestartMode.RESTART_CRD , 2, 1, 2, DFLT_PARTITION_COUNT, + new InitIndexing(Integer.class, Integer.class), SCAN, DML); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups2_RestartCoordinator_SqlDml_Persistence() throws Exception { + persistence = true; + + putAllGetAll(RestartMode.RESTART_CRD, 4, 2, 2, 64, + new InitIndexing(Integer.class, Integer.class), SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllGetAll_ClientServer_Backups1_RestartCoordinator_SqlDml() throws Exception { + putAllGetAll(RestartMode.RESTART_CRD, 2, 1, 1, 64, + new InitIndexing(Integer.class, Integer.class), SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_ClientServer_Backups2_Sql() throws Exception { + updateNObjectsTest(7, 3, 2, 2, DFLT_PARTITION_COUNT, DFLT_TEST_TIME, + new InitIndexing(Integer.class, Integer.class), SQL, DML, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testUpdate_N_Objects_ClientServer_Backups1_Sql_Persistence() throws Exception { + persistence = true; + + updateNObjectsTest(10, 2, 1, 1, DFLT_PARTITION_COUNT, DFLT_TEST_TIME, + new InitIndexing(Integer.class, Integer.class), SQL, DML, RestartMode.RESTART_CRD); + } + + /** + * @throws Exception If failed. + */ + public void testSqlReadInProgressCoordinatorFails() throws Exception { + readInProgressCoordinatorFails(false, false, PESSIMISTIC, REPEATABLE_READ, SQL, DML, new InitIndexing(Integer.class, Integer.class)); + } + + /** + * @throws Exception If failed. + */ + public void testSqlReadInsideTxInProgressCoordinatorFails() throws Exception { + readInProgressCoordinatorFails(false, true, PESSIMISTIC, REPEATABLE_READ, SQL, DML, new InitIndexing(Integer.class, Integer.class)); + } + + /** + * @throws Exception If failed. + */ + public void testSqlReadInProgressCoordinatorFails_ReadDelay() throws Exception { + readInProgressCoordinatorFails(true, false, PESSIMISTIC, REPEATABLE_READ, SQL, DML, new InitIndexing(Integer.class, Integer.class)); + } + + /** + * @throws Exception If failed. + */ + public void testSqlReadInsideTxInProgressCoordinatorFails_ReadDelay() throws Exception { + readInProgressCoordinatorFails(true, true, PESSIMISTIC, REPEATABLE_READ, SQL, DML, new InitIndexing(Integer.class, Integer.class)); + } + + /** + * @throws Exception If failed. + */ + public void testReadInProgressCoordinatorFailsSimple_FromServer() throws Exception { + readInProgressCoordinatorFailsSimple(false, new InitIndexing(Integer.class, Integer.class), SQL, DML); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlQueriesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlQueriesTest.java new file mode 100644 index 0000000000000..e0b4a240e3f1b --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlQueriesTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** */ +public class CacheMvccPartitionedSqlQueriesTest extends CacheMvccSqlQueriesAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return PARTITIONED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesTest.java new file mode 100644 index 0000000000000..199cfad017b56 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** */ +public class CacheMvccPartitionedSqlTxQueriesTest extends CacheMvccSqlTxQueriesAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return PARTITIONED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesWithReducerTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesWithReducerTest.java new file mode 100644 index 0000000000000..03de543f307d3 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccPartitionedSqlTxQueriesWithReducerTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** */ +public class CacheMvccPartitionedSqlTxQueriesWithReducerTest extends CacheMvccSqlTxQueriesWithReducerAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return PARTITIONED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedBackupsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedBackupsTest.java new file mode 100644 index 0000000000000..02de0a346f304 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedBackupsTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.REPLICATED; + +/** */ +public class CacheMvccReplicatedBackupsTest extends CacheMvccBackupsAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return REPLICATED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSelectForUpdateQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSelectForUpdateQueryTest.java new file mode 100644 index 0000000000000..a45831942b889 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSelectForUpdateQueryTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.REPLICATED; + +/** */ +public class CacheMvccReplicatedSelectForUpdateQueryTest extends CacheMvccSelectForUpdateQueryAbstractTest { + /** {@inheritDoc} */ + public CacheMode cacheMode() { + return REPLICATED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlCoordinatorFailoverTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlCoordinatorFailoverTest.java new file mode 100644 index 0000000000000..2f72bcea7fe32 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlCoordinatorFailoverTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +/** + * SQL Mvcc coordinator failover test for replicated caches. + */ +public class CacheMvccReplicatedSqlCoordinatorFailoverTest extends CacheMvccAbstractSqlCoordinatorFailoverTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.REPLICATED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlQueriesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlQueriesTest.java new file mode 100644 index 0000000000000..ba8a5c3659ee4 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlQueriesTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.REPLICATED; + +/** */ +public class CacheMvccReplicatedSqlQueriesTest extends CacheMvccSqlQueriesAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return REPLICATED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java new file mode 100644 index 0000000000000..bde2c5dd6d9e3 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java @@ -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. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import java.util.List; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.transactions.Transaction; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +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; + +/** */ +public class CacheMvccReplicatedSqlTxQueriesTest extends CacheMvccSqlTxQueriesAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return REPLICATED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + ccfgs = null; + ccfg = null; + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedJoinPartitionedClient() throws Exception { + checkReplicatedJoinPartitioned(true); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedJoinPartitionedServer() throws Exception { + checkReplicatedJoinPartitioned(false); + } + + /** + * @throws Exception If failed. + */ + public void checkReplicatedJoinPartitioned(boolean client) throws Exception { + ccfgs = new CacheConfiguration[] { + cacheConfiguration(REPLICATED, FULL_SYNC, 0, DFLT_PARTITION_COUNT) + .setName("int") + .setIndexedTypes(Integer.class, Integer.class), + cacheConfiguration(PARTITIONED, FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, + CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class), + cacheConfiguration(REPLICATED, FULL_SYNC, 0, DFLT_PARTITION_COUNT) + .setName("target") + .setIndexedTypes(Integer.class, Integer.class) + }; + + startGridsMultiThreaded(3); + + this.client = true; + + startGrid(3); + + Ignite node = client ? grid(3) : grid(0); + + List> r; + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + r = runSql(node, "INSERT INTO \"int\".Integer(_key, _val) VALUES (1,1), (2,2), (3,3)"); + + assertEquals(3L, r.get(0).get(0)); + + tx.commit(); + } + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + r = runSql(node, "INSERT INTO \"default\".MvccTestSqlIndexValue(_key, idxVal1) " + + "VALUES (1,10), (2, 20), (3, 30)"); + + assertEquals(3L, r.get(0).get(0)); + + tx.commit(); + } + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + r = runSql(node, "INSERT INTO \"target\".Integer(_key, _val) " + + "SELECT a._key, a.idxVal1*b._val FROM \"default\".MvccTestSqlIndexValue a " + + "JOIN \"int\".Integer b ON a._key = b._key"); + + assertEquals(3L, r.get(0).get(0)); + + tx.commit(); + } + + for (int n = 0; n < 3; ++n) { + node = grid(n); + + r = runSqlLocal(node, "SELECT _key, _val FROM \"target\".Integer ORDER BY _key"); + + assertEquals(3L, r.size()); + + assertEquals(1, r.get(0).get(0)); + assertEquals(2, r.get(1).get(0)); + assertEquals(3, r.get(2).get(0)); + + assertEquals(10, r.get(0).get(1)); + assertEquals(40, r.get(1).get(1)); + assertEquals(90, r.get(2).get(1)); + } + } + + /** + * + * @throws Exception If failed. + */ + public void testReplicatedAndPartitionedUpdateSingleTransaction() throws Exception { + ccfgs = new CacheConfiguration[] { + cacheConfiguration(REPLICATED, FULL_SYNC, 0, DFLT_PARTITION_COUNT) + .setName("rep") + .setIndexedTypes(Integer.class, Integer.class), + cacheConfiguration(PARTITIONED, FULL_SYNC, 0, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class) + .setName("part"), + }; + + startGridsMultiThreaded(3); + + client = true; + + startGrid(3); + + Random rnd = ThreadLocalRandom.current(); + + Ignite node = grid(rnd.nextInt(4)); + + List> r; + + Cache repCache = node.cache("rep"); + + repCache.put(1, 1); + repCache.put(2, 2); + repCache.put(3, 3); + + Cache partCache = node.cache("part"); + + partCache.put(1, new MvccTestSqlIndexValue(1)); + partCache.put(2, new MvccTestSqlIndexValue(2)); + partCache.put(3, new MvccTestSqlIndexValue(3)); + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + r = runSql(node, "UPDATE \"rep\".Integer SET _val = _key * 10"); + + assertEquals(3L, r.get(0).get(0)); + + r = runSql(node, "UPDATE \"part\".MvccTestSqlIndexValue SET idxVal1 = _key * 10"); + + assertEquals(3L, r.get(0).get(0)); + + tx.commit(); + } + + r = runSql(node, "SELECT COUNT(1) FROM \"rep\".Integer r JOIN \"part\".MvccTestSqlIndexValue p" + + " ON r._key = p._key WHERE r._val = p.idxVal1"); + + assertEquals(3L, r.get(0).get(0)); + + for (int n = 0; n < 3; ++n) { + node = grid(n); + + r = runSqlLocal(node, "SELECT _key, _val FROM \"rep\".Integer ORDER BY _key"); + + assertEquals(3L, r.size()); + + assertEquals(1, r.get(0).get(0)); + assertEquals(2, r.get(1).get(0)); + assertEquals(3, r.get(2).get(0)); + + assertEquals(10, r.get(0).get(1)); + assertEquals(20, r.get(1).get(1)); + assertEquals(30, r.get(2).get(1)); + } + } + + /** + * Run query. + * + * @param node Node. + * @param sqlText Query. + * @return Results. + */ + private List> runSql(Ignite node, String sqlText) { + GridQueryProcessor qryProc = ((IgniteEx)node).context().query(); + + return qryProc.querySqlFields(new SqlFieldsQuery(sqlText), false).getAll(); + } + + /** + * Run query locally. + * + * @param node Node. + * @param sqlText Query. + * @return Results. + */ + private List> runSqlLocal(Ignite node, String sqlText) { + GridQueryProcessor qryProc = ((IgniteEx)node).context().query(); + + return qryProc.querySqlFields(new SqlFieldsQuery(sqlText).setLocal(true), false).getAll(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesWithReducerTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesWithReducerTest.java new file mode 100644 index 0000000000000..173c43f126d60 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesWithReducerTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.mvcc; + +import org.apache.ignite.cache.CacheMode; + +import static org.apache.ignite.cache.CacheMode.REPLICATED; + +/** */ +public class CacheMvccReplicatedSqlTxQueriesWithReducerTest extends CacheMvccSqlTxQueriesWithReducerAbstractTest { + /** {@inheritDoc} */ + protected CacheMode cacheMode() { + return REPLICATED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentJdbcTransactionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentJdbcTransactionTest.java new file mode 100644 index 0000000000000..7272def94632f --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccScanQueryWithConcurrentJdbcTransactionTest.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.internal.processors.cache.mvcc; + +/** + * + */ +public class CacheMvccScanQueryWithConcurrentJdbcTransactionTest extends + CacheMvccScanQueryWithConcurrentTransactionTest { + /** {@inheritDoc} */ + @Override boolean jdbcTx() { + return true; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java new file mode 100644 index 0000000000000..8492d2a067db3 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSelectForUpdateQueryAbstractTest.java @@ -0,0 +1,374 @@ +/* + * 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.mvcc; + +import java.sql.Connection; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.query.FieldsQueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +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.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.internal.processors.cache.index.AbstractSchemaSelfTest.connect; +import static org.apache.ignite.internal.processors.cache.index.AbstractSchemaSelfTest.execute; + +/** + * Test for {@code SELECT FOR UPDATE} queries. + */ +public abstract class CacheMvccSelectForUpdateQueryAbstractTest extends CacheMvccAbstractTest { + /** */ + private static final int CACHE_SIZE = 50; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + disableScheduledVacuum = getName().equals("testSelectForUpdateAfterAbortedTx"); + + startGrids(3); + + CacheConfiguration seg = new CacheConfiguration("segmented*"); + + seg.setCacheMode(cacheMode()); + + if (seg.getCacheMode() == PARTITIONED) + seg.setQueryParallelism(4); + + grid(0).addCacheConfiguration(seg); + + try (Connection c = connect(grid(0))) { + execute(c, "create table person (id int primary key, firstName varchar, lastName varchar) " + + "with \"atomicity=transactional,cache_name=Person\""); + + execute(c, "create table person_seg (id int primary key, firstName varchar, lastName varchar) " + + "with \"atomicity=transactional,cache_name=PersonSeg,template=segmented\""); + + try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ)) { + + for (int i = 1; i <= CACHE_SIZE; i++) { + execute(c, "insert into person(id, firstName, lastName) values(" + i + ",'" + i + "','" + i + "')"); + + execute(c, "insert into person_seg(id, firstName, lastName) " + + "values(" + i + ",'" + i + "','" + i + "')"); + } + + tx.commit(); + } + } + + AffinityTopologyVersion curVer = grid(0).context().cache().context().exchange().readyAffinityVersion(); + + AffinityTopologyVersion nextVer = curVer.nextMinorVersion(); + + // Let's wait for rebalance to complete. + for (int i = 0; i < 3; i++) { + IgniteEx node = grid(i); + + IgniteInternalFuture fut = + node.context().cache().context().exchange().affinityReadyFuture(nextVer); + + if (fut != null) + fut.get(); + } + } + + /** + * + */ + public void testSelectForUpdateDistributed() throws Exception { + doTestSelectForUpdateDistributed("Person", false); + } + + + /** + * + */ + public void testSelectForUpdateLocal() throws Exception { + doTestSelectForUpdateLocal("Person", false); + } + + /** + * + * @throws Exception If failed. + */ + public void testSelectForUpdateOutsideTx() throws Exception { + doTestSelectForUpdateDistributed("Person", true); + } + + /** + * + * @throws Exception If failed. + */ + public void testSelectForUpdateOutsideTxLocal() throws Exception { + doTestSelectForUpdateLocal("Person", true); + } + + /** + * @param cacheName Cache name. + * @param outsideTx Whether select is executed outside transaction + * @throws Exception If failed. + */ + void doTestSelectForUpdateLocal(String cacheName, boolean outsideTx) throws Exception { + Ignite node = grid(0); + + IgniteCache cache = node.cache(cacheName); + + Transaction ignored = outsideTx ? null : node.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ); + + try { + SqlFieldsQuery qry = new SqlFieldsQuery("select id, * from " + tableName(cache) + " order by id for update") + .setLocal(true); + + FieldsQueryCursor> query = cache.query(qry); + + List> res = query.getAll(); + + List keys = new ArrayList<>(); + + for (List r : res) + keys.add((Integer)r.get(0)); + + checkLocks(cacheName, keys, !outsideTx); + } + finally { + U.close(ignored, log); + } + } + + /** + * @param cacheName Cache name. + * @param outsideTx Whether select is executed outside transaction + * @throws Exception If failed. + */ + void doTestSelectForUpdateDistributed(String cacheName, boolean outsideTx) throws Exception { + Ignite node = grid(0); + + IgniteCache cache = node.cache(cacheName); + + Transaction ignored = outsideTx ? null : node.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ); + + try { + SqlFieldsQuery qry = new SqlFieldsQuery("select id, * from " + tableName(cache) + " order by id for update") + .setPageSize(10); + + FieldsQueryCursor> query = cache.query(qry); + + List> res = query.getAll(); + + List keys = new ArrayList<>(); + + for (List r : res) + keys.add((Integer)r.get(0)); + + checkLocks(cacheName, keys, !outsideTx); + } + finally { + U.close(ignored, log); + } + } + + /** + * + */ + public void testSelectForUpdateWithUnion() { + assertQueryThrows("select id from person union select 1 for update", + "SELECT UNION FOR UPDATE is not supported."); + } + + /** + * + */ + public void testSelectForUpdateWithJoin() { + assertQueryThrows("select p1.id from person p1 join person p2 on p1.id = p2.id for update", + "SELECT FOR UPDATE with joins is not supported."); + } + + /** + * + */ + public void testSelectForUpdateWithLimit() { + assertQueryThrows("select id from person limit 0,5 for update", + "LIMIT/OFFSET clauses are not supported for SELECT FOR UPDATE."); + } + + /** + * + */ + public void testSelectForUpdateWithGroupings() { + assertQueryThrows("select count(*) from person for update", + "SELECT FOR UPDATE with aggregates and/or GROUP BY is not supported."); + + assertQueryThrows("select lastName, count(*) from person group by lastName for update", + "SELECT FOR UPDATE with aggregates and/or GROUP BY is not supported."); + } + + /** + * @throws Exception If failed. + */ + public void testSelectForUpdateAfterAbortedTx() throws Exception { + assert disableScheduledVacuum; + + Ignite node = grid(0); + + IgniteCache cache = node.cache("Person"); + + List> res; + + try (Transaction tx = node.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ)) { + + res = cache.query(new SqlFieldsQuery("update person set lastName=UPPER(lastName)")).getAll(); + + assertEquals((long)CACHE_SIZE, res.get(0).get(0)); + + tx.rollback(); + } + + try (Transaction tx = node.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ)) { + + res = cache.query(new SqlFieldsQuery("select id, * from person order by id for update")).getAll(); + + assertEquals(CACHE_SIZE, res.size()); + + List keys = new ArrayList<>(); + + for (List r : res) + keys.add((Integer)r.get(0)); + + checkLocks("Person", keys, true); + + tx.rollback(); + } + } + + /** + * Check that an attempt to get a lock on any key from given list fails by timeout. + * + * @param cacheName Cache name to check. + * @param keys Keys to check. + * @param locked Whether the key is locked + * @throws Exception if failed. + */ + @SuppressWarnings({"ThrowableNotThrown", "unchecked"}) + private void checkLocks(String cacheName, List keys, boolean locked) throws Exception { + Ignite node = ignite(2); + IgniteCache cache = node.cache(cacheName); + + List> calls = new ArrayList<>(); + + for (int key : keys) { + calls.add(GridTestUtils.runAsync(new Callable() { + /** {@inheritDoc} */ + @Override public Integer call() { + try (Transaction ignored = node.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ)) { + List> res = cache + .query( + new SqlFieldsQuery("select * from " + tableName(cache) + + " where id = " + key + " for update").setTimeout(1, TimeUnit.SECONDS) + ) + .getAll(); + + return (Integer)res.get(0).get(0); + } + } + })); + } + + for (IgniteInternalFuture fut : calls) { + if (!locked) + fut.get(TX_TIMEOUT); + else { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + try { + return fut.get(TX_TIMEOUT); + } + catch (IgniteCheckedException e) { + if (X.hasCause(e, CacheException.class)) + throw X.cause(e, CacheException.class); + + throw e; + } + } + }, CacheException.class, "IgniteTxTimeoutCheckedException"); + } + } + } + + /** + * @param cache Cache. + * @return Name of the table contained by this cache. + */ + @SuppressWarnings("unchecked") + private static String tableName(IgniteCache cache) { + return ((Collection)cache.getConfiguration(CacheConfiguration.class).getQueryEntities()) + .iterator().next().getTableName(); + } + + /** + * Test that query throws exception with expected message. + * @param qry SQL. + * @param exMsg Expected message. + */ + private void assertQueryThrows(String qry, String exMsg) { + assertQueryThrows(qry, exMsg, false); + + assertQueryThrows(qry, exMsg, true); + } + + /** + * Test that query throws exception with expected message. + * @param qry SQL. + * @param exMsg Expected message. + * @param loc Local query flag. + */ + private void assertQueryThrows(String qry, String exMsg, boolean loc) { + Ignite node = grid(0); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() { + return node.cache("Person").query(new SqlFieldsQuery(qry).setLocal(loc)).getAll(); + } + }, IgniteSQLException.class, exMsg); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentJdbcTransactionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentJdbcTransactionTest.java new file mode 100644 index 0000000000000..437195f205fe2 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeWithConcurrentJdbcTransactionTest.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.cache.mvcc; + +/** + * + */ +public class CacheMvccSizeWithConcurrentJdbcTransactionTest extends CacheMvccSizeWithConcurrentTransactionTest { + /** {@inheritDoc} */ + @Override boolean jdbcTx() { + return true; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java new file mode 100644 index 0000000000000..4ad667b9defb3 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java @@ -0,0 +1,1611 @@ +/* + * 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.mvcc; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.lang.GridInClosure3; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.transactions.Transaction; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL_SUM; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.PUT; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * TODO IGNITE-6739: text/spatial indexes with mvcc. + * TODO IGNITE-6739: indexingSpi with mvcc. + * TODO IGNITE-6739: setQueryParallelism with mvcc. + * TODO IGNITE-6739: dynamic index create. + */ +@SuppressWarnings("unchecked") +public abstract class CacheMvccSqlQueriesAbstractTest extends CacheMvccAbstractTest { + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_WithRemoves_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_SingleNode_Persistence() throws Exception { + persistence = true; + + testAccountsTxSql_SingleNode(); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSumSql_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL_SUM, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_WithRemoves_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_WithRemoves_SingleNode_Persistence() throws Exception { + persistence = true; + + testAccountsTxSql_WithRemoves_SingleNode(); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxSql_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, PUT); + } + + /** + * @throws Exception If failed. + */ + public void testUpdateSingleValue_SingleNode() throws Exception { + updateSingleValue(true, false); + } + + /** + * @throws Exception If failed. + */ + public void testUpdateSingleValue_LocalQuery_SingleNode() throws Exception { + updateSingleValue(true, true); + } + + /** + * @throws Exception If failed. + */ + public void testUpdateSingleValue_ClientServer() throws Exception { + updateSingleValue(false, false); + } + + /** + * @param singleNode {@code True} for test with single node. + * @param locQry Local query flag. + * @throws Exception If failed. + */ + private void updateSingleValue(boolean singleNode, final boolean locQry) throws Exception { + final int VALS = 100; + + final int writers = 4; + + final int readers = 4; + + final int INC_BY = 110; + + final IgniteInClosure> init = new IgniteInClosure>() { + @Override public void apply(IgniteCache cache) { + Map vals = new HashMap<>(); + + for (int i = 0; i < VALS; i++) + vals.put(i, new MvccTestSqlIndexValue(i)); + + cache.putAll(vals); + } + }; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + Integer key = rnd.nextInt(VALS); + + cache.cache.invoke(key, new CacheEntryProcessor() { + @Override public Object process(MutableEntry e, Object... args) { + Integer key = e.getKey(); + + MvccTestSqlIndexValue val = e.getValue(); + + int newIdxVal; + + if (val.idxVal1 < INC_BY) { + assertEquals(key.intValue(), val.idxVal1); + + newIdxVal = val.idxVal1 + INC_BY; + } + else { + assertEquals(INC_BY + key, val.idxVal1); + + newIdxVal = key; + } + + e.setValue(new MvccTestSqlIndexValue(newIdxVal)); + + return null; + } + }); + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + List fieldsQrys = new ArrayList<>(); + + fieldsQrys.add( + new SqlFieldsQuery("select _key, idxVal1 from MvccTestSqlIndexValue where idxVal1=?").setLocal(locQry)); + + fieldsQrys.add(new SqlFieldsQuery("select _key, idxVal1 from MvccTestSqlIndexValue where idxVal1=? or idxVal1=?").setLocal(locQry)); + + fieldsQrys.add(new SqlFieldsQuery("select _key, idxVal1 from MvccTestSqlIndexValue where _key=?").setLocal(locQry)); + + List> sqlQrys = new ArrayList<>(); + + sqlQrys.add(new SqlQuery(MvccTestSqlIndexValue.class, "idxVal1=?").setLocal(locQry)); + + sqlQrys.add(new SqlQuery(MvccTestSqlIndexValue.class, "idxVal1=? or idxVal1=?").setLocal(locQry)); + + sqlQrys.add(new SqlQuery(MvccTestSqlIndexValue.class, "_key=?").setLocal(locQry)); + + while (!stop.get()) { + Integer key = rnd.nextInt(VALS); + + int qryIdx = rnd.nextInt(3); + + TestCache cache = randomCache(caches, rnd); + + List> res; + + try { + if (rnd.nextBoolean()) { + SqlFieldsQuery qry = fieldsQrys.get(qryIdx); + + if (qryIdx == 1) + qry.setArgs(key, key + INC_BY); + else + qry.setArgs(key); + + res = cache.cache.query(qry).getAll(); + } + else { + SqlQuery qry = sqlQrys.get(qryIdx); + + if (qryIdx == 1) + qry.setArgs(key, key + INC_BY); + else + qry.setArgs(key); + + res = new ArrayList<>(); + + for (IgniteCache.Entry e : cache.cache.query(qry).getAll()) { + List row = new ArrayList<>(2); + + row.add(e.getKey()); + row.add(e.getValue().idxVal1); + + res.add(row); + } + } + } + finally { + cache.readUnlock(); + } + + assertTrue(qryIdx == 0 || !res.isEmpty()); + + if (!res.isEmpty()) { + assertEquals(1, res.size()); + + List resVals = res.get(0); + + Integer key0 = (Integer)resVals.get(0); + Integer val0 = (Integer)resVals.get(1); + + assertEquals(key, key0); + assertTrue(val0.equals(key) || val0.equals(key + INC_BY)); + } + } + + if (idx == 0) { + SqlFieldsQuery qry = new SqlFieldsQuery("select _key, idxVal1 from MvccTestSqlIndexValue"); + + TestCache cache = randomCache(caches, rnd); + + List> res; + + try { + res = cache.cache.query(qry).getAll(); + } + finally { + cache.readUnlock(); + } + + assertEquals(VALS, res.size()); + + for (List vals : res) + info("Value: " + vals); + } + } + }; + + int srvs; + int clients; + + if (singleNode) { + srvs = 1; + clients = 0; + } + else { + srvs = 4; + clients = 2; + } + + readWriteTest( + null, + srvs, + clients, + 0, + DFLT_PARTITION_COUNT, + writers, + readers, + DFLT_TEST_TIME, + new InitIndexing(Integer.class, MvccTestSqlIndexValue.class), + init, + writer, + reader); + + for (Ignite node : G.allGrids()) + checkActiveQueriesCleanup(node); + } + + /** + * @throws Exception If failed. + */ + public void testJoinTransactional_SingleNode() throws Exception { + joinTransactional(true, false); + } + + /** + * @throws Exception If failed. + */ + public void testJoinTransactional_ClientServer() throws Exception { + joinTransactional(false, false); + } + + /** + * @throws Exception If failed. + */ + public void testJoinTransactional_DistributedJoins_ClientServer() throws Exception { + joinTransactional(false, true); + } + + /** + * @param singleNode {@code True} for test with single node. + * @param distributedJoin {@code True} to test distributed joins. + * @throws Exception If failed. + */ + private void joinTransactional(boolean singleNode, final boolean distributedJoin) throws Exception { + final int KEYS = 100; + + final int writers = 4; + + final int readers = 4; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + try { + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + Integer key = rnd.nextInt(KEYS); + + JoinTestChildKey childKey = new JoinTestChildKey(key); + + JoinTestChild child = (JoinTestChild)cache.cache.get(childKey); + + if (child == null) { + Integer parentKey = distributedJoin ? key + 100 : key; + + child = new JoinTestChild(parentKey); + + cache.cache.put(childKey, child); + + JoinTestParent parent = new JoinTestParent(parentKey); + + cache.cache.put(new JoinTestParentKey(parentKey), parent); + } + else { + cache.cache.remove(childKey); + + cache.cache.remove(new JoinTestParentKey(child.parentId)); + } + + tx.commit(); + } + + cnt++; + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + List qrys = new ArrayList<>(); + + qrys.add(new SqlFieldsQuery("select c.parentId, p.id from " + + "JoinTestChild c left outer join JoinTestParent p on (c.parentId = p.id)"). + setDistributedJoins(distributedJoin)); + + qrys.add(new SqlFieldsQuery("select c.parentId, p.id from " + + "JoinTestChild c left outer join JoinTestParent p on (c.parentId = p.id) where p.id = 10"). + setDistributedJoins(distributedJoin)); + + qrys.add(new SqlFieldsQuery("select c.parentId, p.id from " + + "JoinTestChild c left outer join JoinTestParent p on (c.parentId = p.id) where p.id != 10"). + setDistributedJoins(distributedJoin)); + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + for (SqlFieldsQuery qry : qrys) { + List> res = cache.cache.query(qry).getAll(); + + if (!res.isEmpty()) { + for (List resRow : res) { + Integer parentId = (Integer)resRow.get(1); + + assertNotNull(parentId); + } + } + } + } + finally { + cache.readUnlock(); + } + } + + if (idx == 0) { + TestCache cache = randomCache(caches, rnd); + + try { + List> res = cache.cache.query(qrys.get(0)).getAll(); + + info("Reader finished, result: " + res); + } + finally { + cache.readUnlock(); + } + } + } + }; + + int srvs; + int clients; + + if (singleNode) { + srvs = 1; + clients = 0; + } + else { + srvs = 4; + clients = 2; + } + + readWriteTest( + null, + srvs, + clients, + 0, + DFLT_PARTITION_COUNT, + writers, + readers, + DFLT_TEST_TIME, + new InitIndexing(JoinTestParentKey.class, JoinTestParent.class, + JoinTestChildKey.class, JoinTestChild.class), + null, + writer, + reader); + } + + /** + * @throws Exception If failed. + */ + public void testJoinTransactional_DistributedJoins_ClientServer2() throws Exception { + final int KEYS = 100; + + final int writers = 1; + + final int readers = 4; + + final int CHILDREN_CNT = 10; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + IgniteTransactions txs = cache.cache.unwrap(Ignite.class).transactions(); + + try { + try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { + Integer key = rnd.nextInt(KEYS); + + JoinTestParentKey parentKey = new JoinTestParentKey(key); + + JoinTestParent parent = (JoinTestParent)cache.cache.get(parentKey); + + if (parent == null) { + for (int i = 0; i < CHILDREN_CNT; i++) + cache.cache.put(new JoinTestChildKey(key * 10_000 + i), new JoinTestChild(key)); + + cache.cache.put(parentKey, new JoinTestParent(key)); + } + else { + for (int i = 0; i < CHILDREN_CNT; i++) + cache.cache.remove(new JoinTestChildKey(key * 10_000 + i)); + + cache.cache.remove(parentKey); + } + + tx.commit(); + } + + cnt++; + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + SqlFieldsQuery qry = new SqlFieldsQuery("select c.parentId, p.id from " + + "JoinTestChild c left outer join JoinTestParent p on (c.parentId = p.id) where p.id=?"). + setDistributedJoins(true); + + int cnt = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + qry.setArgs(rnd.nextInt(KEYS)); + + try { + List> res = cache.cache.query(qry).getAll(); + + if (!res.isEmpty()) + assertEquals(CHILDREN_CNT, res.size()); + + cnt++; + } + finally { + cache.readUnlock(); + } + } + + info("Reader finished, read count: " + cnt); + } + }; + + readWriteTest( + null, + 4, + 2, + 0, + DFLT_PARTITION_COUNT, + writers, + readers, + DFLT_TEST_TIME, + new InitIndexing(JoinTestParentKey.class, JoinTestParent.class, + JoinTestChildKey.class, JoinTestChild.class), + null, + writer, + reader); + } + + /** + * @throws Exception If failed. + */ + public void testDistributedJoinSimple() throws Exception { + startGridsMultiThreaded(4); + + Ignite srv0 = ignite(0); + + int[] backups = {0, 1, 2}; + + for (int b : backups) { + IgniteCache cache = srv0.createCache( + cacheConfiguration(cacheMode(), FULL_SYNC, b, DFLT_PARTITION_COUNT). + setIndexedTypes(JoinTestParentKey.class, JoinTestParent.class, JoinTestChildKey.class, JoinTestChild.class)); + + int cntr = 0; + + int expCnt = 0; + + for (int i = 0; i < 10; i++) { + JoinTestParentKey parentKey = new JoinTestParentKey(i); + + cache.put(parentKey, new JoinTestParent(i)); + + for (int c = 0; c < i; c++) { + JoinTestChildKey childKey = new JoinTestChildKey(cntr++); + + cache.put(childKey, new JoinTestChild(i)); + + expCnt++; + } + } + + SqlFieldsQuery qry = new SqlFieldsQuery("select c.parentId, p.id from " + + "JoinTestChild c join JoinTestParent p on (c.parentId = p.id)"). + setDistributedJoins(true); + + Map resMap = new HashMap<>(); + + List> res = cache.query(qry).getAll(); + + assertEquals(expCnt, res.size()); + + for (List resRow : res) { + Integer parentId = (Integer)resRow.get(0); + + Integer cnt = resMap.get(parentId); + + if (cnt == null) + resMap.put(parentId, 1); + else + resMap.put(parentId, cnt + 1); + } + + for (int i = 1; i < 10; i++) + assertEquals(i, (Object)resMap.get(i)); + + srv0.destroyCache(cache.getName()); + } + } + + /** + * @throws Exception If failed. + */ + public void testCacheRecreate() throws Exception { + cacheRecreate(new InitIndexing(Integer.class, MvccTestAccount.class)); + } + + /** + * @throws Exception If failed. + */ + public void testCacheRecreateChangeIndexedType() throws Exception { + Ignite srv0 = startGrid(0); + + final int PARTS = 64; + + { + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, PARTS). + setIndexedTypes(Integer.class, MvccTestAccount.class); + + IgniteCache cache = (IgniteCache)srv0.createCache(ccfg); + + for (int k = 0; k < PARTS * 2; k++) { + assertNull(cache.get(k)); + + int vals = k % 3 + 1; + + for (int v = 0; v < vals; v++) + cache.put(k, new MvccTestAccount(v, 1)); + + assertEquals(vals - 1, cache.get(k).val); + } + + assertEquals(PARTS * 2, cache.query(new SqlQuery<>(MvccTestAccount.class, "true")).getAll().size()); + + srv0.destroyCache(cache.getName()); + } + + { + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, PARTS). + setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class); + + IgniteCache cache = (IgniteCache)srv0.createCache(ccfg); + + for (int k = 0; k < PARTS * 2; k++) { + assertNull(cache.get(k)); + + int vals = k % 3 + 1; + + for (int v = 0; v < vals; v++) + cache.put(k, new MvccTestSqlIndexValue(v)); + + assertEquals(vals - 1, cache.get(k).idxVal1); + } + + assertEquals(PARTS * 2, cache.query(new SqlQuery<>(MvccTestSqlIndexValue.class, "true")).getAll().size()); + + srv0.destroyCache(cache.getName()); + } + + { + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, PARTS). + setIndexedTypes(Long.class, Long.class); + + IgniteCache cache = (IgniteCache)srv0.createCache(ccfg); + + for (int k = 0; k < PARTS * 2; k++) { + assertNull(cache.get((long)k)); + + int vals = k % 3 + 1; + + for (int v = 0; v < vals; v++) + cache.put((long)k, (long)v); + + assertEquals((long)(vals - 1), (Object)cache.get((long)k)); + } + + assertEquals(PARTS * 2, cache.query(new SqlQuery<>(Long.class, "true")).getAll().size()); + + srv0.destroyCache(cache.getName()); + } + } + + /** + * @throws Exception If failed. + */ + public void testChangeValueType1() throws Exception { + Ignite srv0 = startGrid(0); + + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class, Integer.class, Integer.class); + + IgniteCache cache = srv0.createCache(ccfg); + + cache.put(1, new MvccTestSqlIndexValue(1)); + cache.put(1, new MvccTestSqlIndexValue(2)); + + checkSingleResult(cache, new SqlFieldsQuery("select idxVal1 from MvccTestSqlIndexValue"), 2); + + cache.put(1, 1); + + assertEquals(0, cache.query(new SqlFieldsQuery("select idxVal1 from MvccTestSqlIndexValue")).getAll().size()); + + checkSingleResult(cache, new SqlFieldsQuery("select _val from Integer"), 1); + + cache.put(1, 2); + + checkSingleResult(cache, new SqlFieldsQuery("select _val from Integer"), 2); + } + + /** + * @throws Exception If failed. + */ + public void testChangeValueType2() throws Exception { + Ignite srv0 = startGrid(0); + + CacheConfiguration ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class, Integer.class, Integer.class); + + IgniteCache cache = srv0.createCache(ccfg); + + cache.put(1, new MvccTestSqlIndexValue(1)); + cache.put(1, new MvccTestSqlIndexValue(2)); + + checkSingleResult(cache, new SqlFieldsQuery("select idxVal1 from MvccTestSqlIndexValue"), 2); + + cache.remove(1); + + assertEquals(0, cache.query(new SqlFieldsQuery("select idxVal1 from MvccTestSqlIndexValue")).getAll().size()); + + cache.put(1, 1); + + assertEquals(0, cache.query(new SqlFieldsQuery("select idxVal1 from MvccTestSqlIndexValue")).getAll().size()); + + checkSingleResult(cache, new SqlFieldsQuery("select _val from Integer"), 1); + + cache.put(1, 2); + + checkSingleResult(cache, new SqlFieldsQuery("select _val from Integer"), 2); + } + + /** + * @throws Exception If failed. + */ + public void testCountTransactional_SingleNode() throws Exception { + countTransactional(true); + } + + /** + * @throws Exception If failed. + */ + public void testCountTransactional_ClientServer() throws Exception { + countTransactional(false); + } + + /** + * @param singleNode {@code True} for test with single node. + * @throws Exception If failed. + */ + private void countTransactional(boolean singleNode) throws Exception { + final int writers = 4; + + final int readers = 4; + + final int THREAD_KEY_RANGE = 100; + + final int VAL_RANGE = 10; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int min = idx * THREAD_KEY_RANGE; + int max = min + THREAD_KEY_RANGE; + + info("Thread range [min=" + min + ", max=" + max + ']'); + + int cnt = 0; + + Set keys = new LinkedHashSet<>(); + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + // Add or remove 10 keys. + if (!keys.isEmpty() && (keys.size() == THREAD_KEY_RANGE || rnd.nextInt(3) == 0)) { + Set rmvKeys = new HashSet<>(); + + for (Integer key : keys) { + rmvKeys.add(key); + + if (rmvKeys.size() == 10) + break; + } + + assertEquals(10, rmvKeys.size()); + + cache.cache.removeAll(rmvKeys); + + keys.removeAll(rmvKeys); + } + else { + TreeMap map = new TreeMap<>(); + + while (map.size() != 10) { + Integer key = rnd.nextInt(min, max); + + if (keys.add(key)) + map.put(key, new MvccTestSqlIndexValue(rnd.nextInt(VAL_RANGE))); + } + + assertEquals(10, map.size()); + + cache.cache.putAll(map); + } + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + List qrys = new ArrayList<>(); + + qrys.add(new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue")); + + qrys.add(new SqlFieldsQuery( + "select count(*) from MvccTestSqlIndexValue where idxVal1 >= 0 and idxVal1 <= " + VAL_RANGE)); + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + for (SqlFieldsQuery qry : qrys) { + List> res = cache.cache.query(qry).getAll(); + + assertEquals(1, res.size()); + + Long cnt = (Long)res.get(0).get(0); + + assertTrue(cnt % 10 == 0); + } + } + finally { + cache.readUnlock(); + } + } + } + }; + + int srvs; + int clients; + + if (singleNode) { + srvs = 1; + clients = 0; + } + else { + srvs = 4; + clients = 2; + } + + readWriteTest( + null, + srvs, + clients, + 0, + DFLT_PARTITION_COUNT, + writers, + readers, + DFLT_TEST_TIME, + new InitIndexing(Integer.class, MvccTestSqlIndexValue.class), + null, + writer, + reader); + } + + /** + * @throws Exception If failed. + */ + public void testMaxMinTransactional_SingleNode() throws Exception { + maxMinTransactional(true); + } + + /** + * @throws Exception If failed. + */ + public void testMaxMinTransactional_ClientServer() throws Exception { + maxMinTransactional(false); + } + + /** + * @param singleNode {@code True} for test with single node. + * @throws Exception If failed. + */ + private void maxMinTransactional(boolean singleNode) throws Exception { + final int writers = 1; + + final int readers = 1; + + final int THREAD_OPS = 10; + + final int OP_RANGE = 10; + + final int THREAD_KEY_RANGE = OP_RANGE * THREAD_OPS; + + GridInClosure3, AtomicBoolean> writer = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int min = idx * THREAD_KEY_RANGE; + + info("Thread range [start=" + min + ']'); + + int cnt = 0; + + boolean add = true; + + int op = 0; + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + int startKey = min + op * OP_RANGE; + + if (add) { + Map vals = new HashMap<>(); + + for (int i = 0; i < 10; i++) { + Integer key = startKey + i + 1; + + vals.put(key, new MvccTestSqlIndexValue(key)); + } + + cache.cache.putAll(vals); + + // info("put " + vals.keySet()); + } + else { + Set rmvKeys = new HashSet<>(); + + for (int i = 0; i < 10; i++) + rmvKeys.add(startKey + i + 1); + + cache.cache.removeAll(rmvKeys); + + // info("remove " + rmvKeys); + } + + if (++op == THREAD_OPS) { + add = !add; + + op = 0; + } + } + finally { + cache.readUnlock(); + } + } + + info("Writer finished, updates: " + cnt); + } + }; + + GridInClosure3, AtomicBoolean> reader = + new GridInClosure3, AtomicBoolean>() { + @Override public void apply(Integer idx, List caches, AtomicBoolean stop) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + List maxQrys = new ArrayList<>(); + List minQrys = new ArrayList<>(); + + maxQrys.add(new SqlFieldsQuery("select max(idxVal1) from MvccTestSqlIndexValue")); + maxQrys.add(new SqlFieldsQuery("select max(idxVal1) from MvccTestSqlIndexValue where idxVal1 >= 0")); + + minQrys.add(new SqlFieldsQuery("select min(idxVal1) from MvccTestSqlIndexValue")); + minQrys.add(new SqlFieldsQuery("select min(idxVal1) from MvccTestSqlIndexValue where idxVal1 >= 0")); + + while (!stop.get()) { + TestCache cache = randomCache(caches, rnd); + + try { + for (SqlFieldsQuery qry : maxQrys) { + List> res = cache.cache.query(qry).getAll(); + + assertEquals(1, res.size()); + + Integer m = (Integer)res.get(0).get(0); + + assertTrue(m == null || m % 10 == 0); + } + + for (SqlFieldsQuery qry : minQrys) { + List> res = cache.cache.query(qry).getAll(); + + assertEquals(1, res.size()); + + Integer m = (Integer)res.get(0).get(0); + + assertTrue(m == null || m % 10 == 1); + } + } + finally { + cache.readUnlock(); + } + } + } + }; + + int srvs; + int clients; + + if (singleNode) { + srvs = 1; + clients = 0; + } + else { + srvs = 4; + clients = 2; + } + + readWriteTest( + null, + srvs, + clients, + 0, + DFLT_PARTITION_COUNT, + writers, + readers, + DFLT_TEST_TIME, + new InitIndexing(Integer.class, MvccTestSqlIndexValue.class), + null, + writer, + reader); + } + + /** + * @throws Exception If failed. + */ + public void testSqlQueriesWithMvcc() throws Exception { + Ignite srv0 = startGrid(0); + + IgniteCache cache = (IgniteCache)srv0.createCache( + cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class)); + + for (int i = 0; i < 10; i++) + cache.put(i, new MvccTestSqlIndexValue(i)); + + sqlQueriesWithMvcc(cache, true); + + sqlQueriesWithMvcc(cache, false); + + // TODO IGNITE-8031 +// startGrid(1); +// +// awaitPartitionMapExchange(); +// +// sqlQueriesWithMvcc(cache, false); + } + + /** + * @param cache Cache. + * @param loc Local query flag. + */ + private void sqlQueriesWithMvcc(IgniteCache cache, boolean loc) { + assertEquals(10, + cache.query(new SqlQuery<>(MvccTestSqlIndexValue.class, "true").setLocal(loc)).getAll().size()); + + assertEquals(10, + cache.query(new SqlFieldsQuery("select idxVal1 from MvccTestSqlIndexValue").setLocal(loc)).getAll().size()); + + assertEquals(10, + cache.query(new SqlFieldsQuery("" + + "select (select count (*) from MvccTestSqlIndexValue where idxVal1 = t1.idxVal1) as c1," + + " (select 0 from dual) as c2" + + " from MvccTestSqlIndexValue as t1" + + " join (select * from MvccTestSqlIndexValue) as t2 on t1.idxVal1 = t2.idxVal1").setLocal(loc)).getAll().size()); + + checkSingleResult(cache, + new SqlFieldsQuery("select max(idxVal1) from MvccTestSqlIndexValue").setLocal(loc), 9); + + checkSingleResult(cache, + new SqlFieldsQuery("select max(idxVal1) from MvccTestSqlIndexValue where idxVal1 > 0").setLocal(loc), 9); + + checkSingleResult(cache, + new SqlFieldsQuery("select max(idxVal1) from MvccTestSqlIndexValue where idxVal1 < 5").setLocal(loc), 4); + + checkSingleResult(cache, + new SqlFieldsQuery("select min(idxVal1) from MvccTestSqlIndexValue").setLocal(loc), 0); + + checkSingleResult(cache, + new SqlFieldsQuery("select min(idxVal1) from MvccTestSqlIndexValue where idxVal1 < 100").setLocal(loc), 0); + + checkSingleResult(cache, + new SqlFieldsQuery("select min(idxVal1) from MvccTestSqlIndexValue where idxVal1 < 5").setLocal(loc), 0); + + checkSingleResult(cache, + new SqlFieldsQuery("select min(idxVal1) from MvccTestSqlIndexValue where idxVal1 > 5").setLocal(loc), 6); + + checkSingleResult(cache, + new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue").setLocal(loc), 10L); + + checkSingleResult(cache, + new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue where idxVal1 >= 0").setLocal(loc), 10L); + + checkSingleResult(cache, + new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue where idxVal1 >= 0 and idxVal1 < 100").setLocal(loc), 10L); + + checkSingleResult(cache, + new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue where idxVal1 >0 and idxVal1 < 5").setLocal(loc), 4L); + + checkSingleResult(cache, + new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue where idxVal1 >= 1").setLocal(loc), 9L); + + checkSingleResult(cache, + new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue where idxVal1 > 100").setLocal(loc), 0L); + + checkSingleResult(cache, + new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue where idxVal1 = 1").setLocal(loc), 1L); + } + + /** + * @param cache Cache. + * @param qry Query. + * @param exp Expected value. + */ + private void checkSingleResult(IgniteCache cache, SqlFieldsQuery qry, Object exp) { + List> res = cache.query(qry).getAll(); + + assertEquals(1, res.size()); + + List row = res.get(0); + + assertEquals(1, row.size()); + + assertEquals(exp, row.get(0)); + } + + /** + * @throws Exception If failed. + */ + public void testSqlSimple() throws Exception { + startGrid(0); + + for (int i = 0; i < 4; i++) + sqlSimple(i * 512); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 5; i++) + sqlSimple(rnd.nextInt(2048)); + } + + /** + * @param inlineSize Inline size. + * @throws Exception If failed. + */ + private void sqlSimple(int inlineSize) throws Exception { + Ignite srv0 = ignite(0); + + IgniteCache cache = (IgniteCache)srv0.createCache( + cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class). + setSqlIndexMaxInlineSize(inlineSize)); + + Map expVals = new HashMap<>(); + + checkValues(expVals, cache); + + cache.put(1, new MvccTestSqlIndexValue(1)); + expVals.put(1, 1); + + checkValues(expVals, cache); + + cache.put(1, new MvccTestSqlIndexValue(2)); + expVals.put(1, 2); + + checkValues(expVals, cache); + + cache.put(2, new MvccTestSqlIndexValue(1)); + expVals.put(2, 1); + cache.put(3, new MvccTestSqlIndexValue(1)); + expVals.put(3, 1); + cache.put(4, new MvccTestSqlIndexValue(1)); + expVals.put(4, 1); + + checkValues(expVals, cache); + + cache.remove(1); + expVals.remove(1); + + checkValues(expVals, cache); + + checkNoValue(1, cache); + + cache.put(1, new MvccTestSqlIndexValue(10)); + expVals.put(1, 10); + + checkValues(expVals, cache); + + checkActiveQueriesCleanup(srv0); + + srv0.destroyCache(cache.getName()); + } + + /** + * @throws Exception If failed. + */ + public void testSqlSimplePutRemoveRandom() throws Exception { + startGrid(0); + + testSqlSimplePutRemoveRandom(0); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 3; i++) + testSqlSimplePutRemoveRandom(rnd.nextInt(2048)); + } + + /** + * @param inlineSize Inline size. + * @throws Exception If failed. + */ + private void testSqlSimplePutRemoveRandom(int inlineSize) throws Exception { + Ignite srv0 = grid(0); + + IgniteCache cache = (IgniteCache) srv0.createCache( + cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT). + setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class). + setSqlIndexMaxInlineSize(inlineSize)); + + Map expVals = new HashMap<>(); + + final int KEYS = 100; + final int VALS = 10; + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + long stopTime = System.currentTimeMillis() + 5_000; + + for (int i = 0; i < 100_000; i++) { + Integer key = rnd.nextInt(KEYS); + + if (rnd.nextInt(5) == 0) { + cache.remove(key); + + expVals.remove(key); + } + else { + Integer val = rnd.nextInt(VALS); + + cache.put(key, new MvccTestSqlIndexValue(val)); + + expVals.put(key, val); + } + + checkValues(expVals, cache); + + if (System.currentTimeMillis() > stopTime) { + info("Stop test, iteration: " + i); + + break; + } + } + + for (int i = 0; i < KEYS; i++) { + if (!expVals.containsKey(i)) + checkNoValue(i, cache); + } + + checkActiveQueriesCleanup(srv0); + + srv0.destroyCache(cache.getName()); + } + + /** + * @param key Key. + * @param cache Cache. + */ + private void checkNoValue(Object key, IgniteCache cache) { + SqlQuery qry; + + qry = new SqlQuery<>(MvccTestSqlIndexValue.class, "_key = ?"); + + qry.setArgs(key); + + List> res = cache.query(qry).getAll(); + + assertTrue(res.isEmpty()); + } + + /** + * @param expVals Expected values. + * @param cache Cache. + */ + private void checkValues(Map expVals, IgniteCache cache) { + SqlFieldsQuery cntQry = new SqlFieldsQuery("select count(*) from MvccTestSqlIndexValue"); + + Long cnt = (Long)cache.query(cntQry).getAll().get(0).get(0); + + assertEquals((long)expVals.size(), (Object)cnt); + + SqlQuery qry; + + qry = new SqlQuery<>(MvccTestSqlIndexValue.class, "true"); + + Map vals = new HashMap<>(); + + for (IgniteCache.Entry e : cache.query(qry).getAll()) + assertNull(vals.put(e.getKey(), e.getValue().idxVal1)); + + assertEquals(expVals, vals); + + qry = new SqlQuery<>(MvccTestSqlIndexValue.class, "_key >= 0"); + + vals = new HashMap<>(); + + for (IgniteCache.Entry e : cache.query(qry).getAll()) + assertNull(vals.put(e.getKey(), e.getValue().idxVal1)); + + assertEquals(expVals, vals); + + qry = new SqlQuery<>(MvccTestSqlIndexValue.class, "idxVal1 >= 0"); + + vals = new HashMap<>(); + + for (IgniteCache.Entry e : cache.query(qry).getAll()) + assertNull(vals.put(e.getKey(), e.getValue().idxVal1)); + + assertEquals(expVals, vals); + + Map> expIdxVals = new HashMap<>(); + + for (Map.Entry e : expVals.entrySet()) { + qry = new SqlQuery<>(MvccTestSqlIndexValue.class, "_key = ?"); + + qry.setArgs(e.getKey()); + + List> res = cache.query(qry).getAll(); + + assertEquals(1, res.size()); + assertEquals(e.getKey(), res.get(0).getKey()); + assertEquals(e.getValue(), (Integer)res.get(0).getValue().idxVal1); + + SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select _key, idxVal1 from MvccTestSqlIndexValue where _key=?"); + fieldsQry.setArgs(e.getKey()); + + List> fieldsRes = cache.query(fieldsQry).getAll(); + + assertEquals(1, fieldsRes.size()); + assertEquals(e.getKey(), fieldsRes.get(0).get(0)); + assertEquals(e.getValue(), fieldsRes.get(0).get(1)); + + Integer val = e.getValue(); + + Set keys = expIdxVals.get(val); + + if (keys == null) + expIdxVals.put(val, keys = new HashSet<>()); + + assertTrue(keys.add(e.getKey())); + } + + for (Map.Entry> expE : expIdxVals.entrySet()) { + qry = new SqlQuery<>(MvccTestSqlIndexValue.class, "idxVal1 = ?"); + qry.setArgs(expE.getKey()); + + vals = new HashMap<>(); + + for (IgniteCache.Entry e : cache.query(qry).getAll()) { + assertNull(vals.put(e.getKey(), e.getValue().idxVal1)); + + assertEquals(expE.getKey(), (Integer)e.getValue().idxVal1); + + assertTrue(expE.getValue().contains(e.getKey())); + } + + assertEquals(expE.getValue().size(), vals.size()); + } + } + + /** + * + */ + static class JoinTestParentKey implements Serializable { + /** */ + private int key; + + /** + * @param key Key. + */ + JoinTestParentKey(int key) { + this.key = key; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + JoinTestParentKey that = (JoinTestParentKey)o; + + return key == that.key; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return key; + } + } + + /** + * + */ + static class JoinTestParent { + /** */ + @QuerySqlField(index = true) + private int id; + + /** + * @param id ID. + */ + JoinTestParent(int id) { + this.id = id; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JoinTestParent.class, this); + } + } + + /** + * + */ + static class JoinTestChildKey implements Serializable { + /** */ + @QuerySqlField(index = true) + private int key; + + /** + * @param key Key. + */ + JoinTestChildKey(int key) { + this.key = key; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + JoinTestChildKey that = (JoinTestChildKey)o; + + return key == that.key; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return key; + } + } + + /** + * + */ + static class JoinTestChild { + /** */ + @QuerySqlField(index = true) + private int parentId; + + /** + * @param parentId Parent ID. + */ + JoinTestChild(int parentId) { + this.parentId = parentId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JoinTestChild.class, this); + } + } + + /** + * + */ + static class MvccTestSqlIndexValue implements Serializable { + /** */ + @QuerySqlField(index = true) + private int idxVal1; + + /** + * @param idxVal1 Indexed value 1. + */ + MvccTestSqlIndexValue(int idxVal1) { + this.idxVal1 = idxVal1; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccTestSqlIndexValue.class, this); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesAbstractTest.java new file mode 100644 index 0000000000000..cb4ade7adf4d1 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesAbstractTest.java @@ -0,0 +1,1832 @@ +/* + * 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.mvcc; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.Phaser; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import javax.cache.Cache; +import javax.cache.CacheException; +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.IgniteCheckedException; +import org.apache.ignite.cache.query.FieldsQueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL_SUM; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.DML; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.testframework.GridTestUtils.runMultiThreaded; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Tests for transactional SQL. + */ +public abstract class CacheMvccSqlTxQueriesAbstractTest extends CacheMvccAbstractTest { + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_SingleNode_SinglePartition() throws Exception { + accountsTxReadAll(1, 0, 0, 1, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_SingleNode_Persistence() throws Exception { + persistence = true; + + testAccountsTxDmlSql_SingleNode(); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSumSql_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL_SUM, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSumSql_WithRemoves_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL_SUM, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSumSql_WithRemoves__ClientServer_Backups0() throws Exception { + accountsTxReadAll(4, 2, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL_SUM, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSumSql_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL_SUM, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_SingleNode() throws Exception { + accountsTxReadAll(1, 0, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_SingleNode_Persistence() throws Exception { + persistence = true; + + testAccountsTxDmlSql_WithRemoves_SingleNode(); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_ClientServer_Backups0() throws Exception { + accountsTxReadAll(4, 2, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_ClientServer_Backups0() throws Exception { + accountsTxReadAll(4, 2, 0, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_ClientServer_Backups0_Persistence() throws Exception { + persistence = true; + + testAccountsTxDmlSql_WithRemoves_ClientServer_Backups0(); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_ClientServer_Backups1() throws Exception { + accountsTxReadAll(3, 0, 1, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_ClientServer_Backups1() throws Exception { + accountsTxReadAll(4, 2, 1, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_ClientServer_Backups1_Persistence() throws Exception { + persistence = true; + + testAccountsTxDmlSql_WithRemoves_ClientServer_Backups1(); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), false, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_WithRemoves_ClientServer_Backups2() throws Exception { + accountsTxReadAll(4, 2, 2, 64, + new InitIndexing(Integer.class, MvccTestAccount.class), true, SQL, DML); + } + + /** + * @throws Exception If failed. + */ + public void testAccountsTxDmlSql_ClientServer_Backups2_Persistence() throws Exception { + persistence = true; + + testAccountsTxDmlSql_ClientServer_Backups2(); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertStaticCache() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (4,4),(5,5),(6,6)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + assertEquals(4, cache.get(4)); + assertEquals(5, cache.get(5)); + assertEquals(6, cache.get(6)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertStaticCacheImplicit() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryDeleteStaticCache() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + qry = new SqlFieldsQuery("DELETE FROM Integer WHERE 1 = 1"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertNull(cache.get(1)); + assertNull(cache.get(2)); + assertNull(cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryFastDeleteStaticCache() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + qry = new SqlFieldsQuery("DELETE FROM Integer WHERE _key = 1"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertNull(cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryFastUpdateStaticCache() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + qry = new SqlFieldsQuery("UPDATE Integer SET _val = 8 WHERE _key = 1"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(8, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryFastDeleteObjectStaticCache() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1, new MvccTestSqlIndexValue(1), + 2, new MvccTestSqlIndexValue(2), + 3, new MvccTestSqlIndexValue(3))); + + assertEquals(new MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new MvccTestSqlIndexValue(3), cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("DELETE FROM MvccTestSqlIndexValue WHERE _key = 1"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertNull(cache.get(1)); + assertEquals(new MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new MvccTestSqlIndexValue(3), cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryFastUpdateObjectStaticCache() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1, new MvccTestSqlIndexValue(1), + 2, new MvccTestSqlIndexValue(2), + 3, new MvccTestSqlIndexValue(3))); + + assertEquals(new MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new MvccTestSqlIndexValue(3), cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE MvccTestSqlIndexValue SET idxVal1 = 8 WHERE _key = 1"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(new MvccTestSqlIndexValue(8), cache.get(1)); + assertEquals(new MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new MvccTestSqlIndexValue(3), cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryDeleteStaticCacheImplicit() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + SqlFieldsQuery qry = new SqlFieldsQuery("DELETE FROM Integer WHERE 1 = 1") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertNull(cache.get(1)); + assertNull(cache.get(2)); + assertNull(cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryUpdateStaticCache() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE Integer SET _val = (_key * 10)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(10, cache.get(1)); + assertEquals(20, cache.get(2)); + assertEquals(30, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryUpdateStaticCacheImplicit() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap(1, 1, 2, 2, 3, 3)); + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE Integer SET _val = (_key * 10)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(10, cache.get(1)); + assertEquals(20, cache.get(2)); + assertEquals(30, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryDeadlock() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + client = true; + + startGridsMultiThreaded(2, 2); + + final CyclicBarrier barrier = new CyclicBarrier(2); + final AtomicInteger idx = new AtomicInteger(); + final AtomicReference ex = new AtomicReference<>(); + + multithreaded(new Runnable() { + @Override public void run() { + int id = idx.getAndIncrement(); + + IgniteEx node = grid(id); + + try { + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + + String qry1 = "INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"; + String qry2 = "INSERT INTO Integer (_key, _val) values (4,4),(5,5),(6,6)"; + + SqlFieldsQuery qry = new SqlFieldsQuery((id % 2) == 0 ? qry1 : qry2); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + cur.getAll(); + } + + barrier.await(); + + qry = new SqlFieldsQuery((id % 2) == 0 ? qry2 : qry1); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + cur.getAll(); + } + + tx.commit(); + } + } + catch (Exception e) { + onException(ex, e); + } + } + }, 2); + + Exception ex0 = ex.get(); + + assertNotNull(ex0); + + if (!X.hasCause(ex0, IgniteTxTimeoutCheckedException.class)) + throw ex0; + } + + /** + * @throws Exception If failed. + */ + public void testQueryDeadlockImplicit() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 0, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + final Phaser phaser = new Phaser(2); + final AtomicReference ex = new AtomicReference<>(); + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + IgniteEx node = grid(0); + + try { + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + cur.getAll(); + } + + awaitPhase(phaser, 2); + + tx.commit(); + } + } + catch (Exception e) { + onException(ex, e); + } + finally { + phaser.arrive(); + } + } + }); + + phaser.arriveAndAwaitAdvance(); + + IgniteEx node = grid(1); + + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + cur.getAll(); + } + catch (Exception e) { + phaser.arrive(); + + onException(ex, e); + } + + phaser.arriveAndAwaitAdvance(); + + Exception ex0 = ex.get(); + + assertNotNull(ex0); + + if (!X.hasCause(ex0, IgniteTxTimeoutCheckedException.class)) + throw ex0; + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertClient() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGrid(0); + + client = true; + + startGrid(1); + + awaitPartitionMapExchange(); + + Ignite checkNode = grid(0); + Ignite updateNode = grid(1); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (4,4),(5,5),(6,6)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + assertEquals(4, cache.get(4)); + assertEquals(5, cache.get(5)); + assertEquals(6, cache.get(6)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertClientImplicit() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGrid(0); + + client = true; + + startGrid(1); + + awaitPartitionMapExchange(); + + Ignite checkNode = grid(0); + Ignite updateNode = grid(1); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertSubquery() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class, Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + awaitPartitionMapExchange(); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1, new MvccTestSqlIndexValue(1), + 2, new MvccTestSqlIndexValue(2), + 3, new MvccTestSqlIndexValue(3))); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val)" + + " SELECT _key * 10, idxVal1 FROM MvccTestSqlIndexValue"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(1, cache0.get(10)); + assertEquals(2, cache0.get(20)); + assertEquals(3, cache0.get(30)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertSubqueryImplicit() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class, Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + awaitPartitionMapExchange(); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1, new MvccTestSqlIndexValue(1), + 2, new MvccTestSqlIndexValue(2), + 3, new MvccTestSqlIndexValue(3))); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val)" + + " SELECT _key * 10, idxVal1 FROM MvccTestSqlIndexValue") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(1, cache0.get(10)); + assertEquals(2, cache0.get(20)); + assertEquals(3, cache0.get(30)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryUpdateSubquery() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class, Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + awaitPartitionMapExchange(); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1, new MvccTestSqlIndexValue(1), + 2, new MvccTestSqlIndexValue(2), + 3, new MvccTestSqlIndexValue(3))); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE MvccTestSqlIndexValue AS t " + + "SET (idxVal1) = (SELECT idxVal1*10 FROM MvccTestSqlIndexValue WHERE t._key = _key)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(10, ((MvccTestSqlIndexValue)cache.get(1)).idxVal1); + assertEquals(20, ((MvccTestSqlIndexValue)cache.get(2)).idxVal1); + assertEquals(30, ((MvccTestSqlIndexValue)cache.get(3)).idxVal1); + } + + /** + * @throws Exception If failed. + */ + public void testQueryUpdateSubqueryImplicit() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class, Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + awaitPartitionMapExchange(); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1, new MvccTestSqlIndexValue(1), + 2, new MvccTestSqlIndexValue(2), + 3, new MvccTestSqlIndexValue(3))); + + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE MvccTestSqlIndexValue AS t " + + "SET (idxVal1) = (SELECT idxVal1*10 FROM MvccTestSqlIndexValue WHERE t._key = _key)") + .setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(10, ((MvccTestSqlIndexValue)cache.get(1)).idxVal1); + assertEquals(20, ((MvccTestSqlIndexValue)cache.get(2)).idxVal1); + assertEquals(30, ((MvccTestSqlIndexValue)cache.get(3)).idxVal1); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertMultithread() throws Exception { + final int THREAD_CNT = 8; + final int BATCH_SIZE = 1000; + final int ROUNDS = 10; + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + client = true; + + startGridsMultiThreaded(2, 2); + + final AtomicInteger seq = new AtomicInteger(); + + multithreaded(new Runnable() { + @Override public void run() { + for (int r = 0; r < ROUNDS; r++) { + StringBuilder bldr = new StringBuilder("INSERT INTO Integer (_key, _val) values "); + + int start = seq.getAndAdd(BATCH_SIZE); + + for (int i = start, end = start + BATCH_SIZE; i < end; i++) { + if (i != start) + bldr.append(','); + + bldr + .append('(') + .append(i) + .append(',') + .append(i) + .append(')'); + } + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery(bldr.toString()).setPageSize(100); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals((long)BATCH_SIZE, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + for (int i = start, end = start + BATCH_SIZE; i < end; i++) + assertEquals(i, cache.get(i)); + } + + } + }, THREAD_CNT); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertUpdateMultithread() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + final Phaser phaser = new Phaser(2); + final AtomicReference ex = new AtomicReference<>(); + + GridCompoundFuture fut = new GridCompoundFuture(); + + fut.add(multithreadedAsync(new Runnable() { + @Override public void run() { + IgniteEx node = grid(0); + + try { + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + cur.getAll(); + } + + awaitPhase(phaser, 2); + + qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (4,4),(5,5),(6,6)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + cur.getAll(); + } + + tx.commit(); + } + } + catch (Exception e) { + onException(ex, e); + } + } + }, 1)); + + fut.add(multithreadedAsync(new Runnable() { + @Override public void run() { + IgniteEx node = grid(1); + + try { + phaser.arriveAndAwaitAdvance(); + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + + cache0.invokeAllAsync(F.asSet(1, 2, 3, 4, 5, 6), new EntryProcessor() { + @Override + public Void process(MutableEntry entry, + Object... arguments) throws EntryProcessorException { + entry.setValue(entry.getValue() * 10); + + return null; + } + }); + + phaser.arrive(); + + tx.commit(); + } + } + catch (Exception e) { + onException(ex, e); + } + } + }, 1)); + + fut.markInitialized(); + + try { + fut.get(TX_TIMEOUT); + } + catch (IgniteCheckedException e) { + onException(ex, e); + } + + Exception ex0 = ex.get(); + + if (ex0 != null) + throw ex0; + + IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME); + + assertEquals(10, cache.get(1)); + assertEquals(20, cache.get(2)); + assertEquals(30, cache.get(3)); + assertEquals(40, cache.get(4)); + assertEquals(50, cache.get(5)); + assertEquals(60, cache.get(6)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertVersionConflict() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1)"); + + try (FieldsQueryCursor> cur = cache.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + final CyclicBarrier barrier = new CyclicBarrier(2); + final AtomicReference ex = new AtomicReference<>(); + + runMultiThreaded(new Runnable() { + @Override public void run() { + IgniteEx node = grid(0); + + try { + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + barrier.await(); + + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry; + + synchronized (barrier) { + qry = new SqlFieldsQuery("SELECT * FROM Integer"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1, cur.getAll().size()); + } + } + + barrier.await(); + + qry = new SqlFieldsQuery("UPDATE Integer SET _val = (_key * 10)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + } + catch (Exception e) { + onException(ex, e); + } + } + }, 2, "tx-thread"); + + IgniteSQLException ex0 = X.cause(ex.get(), IgniteSQLException.class); + + assertNotNull("Exception has not been thrown.", ex0); + assertEquals("Mvcc version mismatch.", ex0.getMessage()); + } + + /** + * @throws Exception If failed. + */ + public void testInsertAndFastDeleteWithoutVersionConflict() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + IgniteCache cache0 = grid(0).cache(DEFAULT_CACHE_NAME); + + try (Transaction tx1 = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + // obtain tx version + cache0.query(new SqlFieldsQuery("select * from Integer where _key = 1")); + + runAsync(() -> { + cache0.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, ?)").setArgs(1, 1)); + }).get(); + + cache0.query(new SqlFieldsQuery("delete from Integer where _key = ?").setArgs(1)); + + tx1.commit(); + } + catch (Exception e) { + e.printStackTrace(); + + fail("Exception is not expected here"); + } + } + + /** + * @throws Exception If failed. + */ + public void testInsertAndFastUpdateWithoutVersionConflict() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + IgniteCache cache0 = grid(0).cache(DEFAULT_CACHE_NAME); + + try (Transaction tx1 = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + // obtain tx version + cache0.query(new SqlFieldsQuery("select * from Integer where _key = 1")); + + runAsync(() -> { + cache0.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, ?)").setArgs(1, 1)); + }).get(); + + cache0.query(new SqlFieldsQuery("update Integer set _val = ? where _key = ?").setArgs(1, 1)); + + tx1.commit(); + } + catch (Exception e) { + e.printStackTrace(); + + fail("Exception is not expected here"); + } + } + + /** + * @throws Exception If failed. + */ + public void testInsertFastUpdateConcurrent() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-9292"); + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(2); + + IgniteCache cache0 = grid(0).cache(DEFAULT_CACHE_NAME); + + try { + for (int i = 0; i < 100; i++) { + int key = i; + CompletableFuture.allOf( + CompletableFuture.runAsync(() -> { + cache0.query(new SqlFieldsQuery("insert into Integer(_key, _val) values(?, ?)").setArgs(key, key)); + }), + CompletableFuture.runAsync(() -> { + cache0.query(new SqlFieldsQuery("update Integer set _val = ? where _key = ?").setArgs(key, key)); + }) + ).get(); + } + } + catch (Exception e) { + e.printStackTrace(); + + fail("Exception is not expected here"); + } + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertRollback() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (4,4),(5,5),(6,6)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.rollback(); + } + + for (int i = 1; i <= 6; i++) + assertTrue(cache.query(new SqlFieldsQuery("SELECT * FROM Integer WHERE _key = 1")).getAll().isEmpty()); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertUpdateSameKeys() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + final Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + qry = new SqlFieldsQuery("UPDATE Integer SET _val = (_key * 10)"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + cur.getAll(); + } + + tx.commit(); + } + + assertEquals(10, cache.get(1)); + assertEquals(20, cache.get(2)); + assertEquals(30, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryInsertUpdateSameKeysInSameOperation() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + final Ignite updateNode = grid(rnd.nextInt(4)); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(1,2),(1,3)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + cache0.query(qry).getAll(); + + tx.commit(); + } + + return null; + } + }, CacheException.class, "Duplicate key during INSERT [key=KeyCacheObjectImpl"); + } + + /** + * @throws Exception If failed. + */ + public void testQueryPendingUpdates() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + final Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)"); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + try (FieldsQueryCursor> cur = cache0.query(qry.setSql("UPDATE Integer SET _val = (_key * 10)"))) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + for (List row : cache0.query(qry.setSql("SELECT _key, _val FROM Integer")).getAll()) { + assertEquals((Integer)row.get(0) * 10, row.get(1)); + } + + try (FieldsQueryCursor> cur = cache0.query(qry.setSql("UPDATE Integer SET _val = 15 where _key = 2"))) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + for (List row : cache0.query(qry.setSql("SELECT _key, _val FROM Integer")).getAll()) { + if ((Integer)row.get(0) == 2) + assertEquals(15, row.get(1)); + else + assertEquals((Integer)row.get(0) * 10, row.get(1)); + } + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + SqlFieldsQuery qry = new SqlFieldsQuery("SELECT _key, _val FROM Integer"); + + assertTrue(cache.query(qry).getAll().isEmpty()); + } + }).get(TX_TIMEOUT); + + cache0.query(qry.setSql("DELETE FROM Integer")).getAll(); + + assertTrue(cache0.query(qry.setSql("SELECT _key, _val FROM Integer")).getAll().isEmpty()); + + assertEquals(3L, cache0.query(qry.setSql("INSERT INTO Integer (_key, _val) values (1,1),(2,2),(3,3)")).getAll().iterator().next().get(0)); + + tx.commit(); + } + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testSelectProducesTransaction() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite node = grid(rnd.nextInt(4)); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) values (1,1),(2,2),(3,3)"); + + try (FieldsQueryCursor> cur = cache.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + SqlFieldsQueryEx qryEx = new SqlFieldsQueryEx("SELECT * FROM MvccTestSqlIndexValue", true); + + qryEx.setAutoCommit(false); + + try (FieldsQueryCursor> cur = cache.query(qryEx)) { + assertEquals(3, cur.getAll().size()); + } + + try (GridNearTxLocal tx = cache.unwrap(IgniteEx.class).context().cache().context().tm().userTx()) { + assertNotNull(tx); + } + } + + /** + * @throws Exception If failed. + */ + public void testRepeatableRead() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + IgniteCache cache = grid(rnd.nextInt(4)).cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache.query( + new SqlFieldsQuery("INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) values (1,1),(2,2),(3,3)"))) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + Ignite node = grid(rnd.nextInt(4)); + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + SqlFieldsQuery qry = new SqlFieldsQuery("SELECT * FROM MvccTestSqlIndexValue"); + + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3, cur.getAll().size()); + } + + runAsync(new Runnable() { + @Override public void run() { + IgniteCache cache = grid(ThreadLocalRandom.current().nextInt(4)) + .cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache.query( + new SqlFieldsQuery("INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) values (4,4),(5,5),(6,6)"))) { + assertEquals(3L, cur.iterator().next().get(0)); + } + } + }).get(TX_TIMEOUT); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3, cur.getAll().size()); + } + } + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(6, cur.getAll().size()); + } + } + + /** + * @throws Exception If failed. + */ + public void testIterator() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGrid(getConfiguration("grid").setMvccVacuumTimeInterval(Integer.MAX_VALUE)); + + Ignite client = startGrid(getConfiguration("client").setClientMode(true)); + + IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); + + cache.put(1, 1); + cache.put(2, 2); + cache.put(3, 3); + cache.put(4, 4); + + List> res; + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + res = cache.query(new SqlFieldsQuery("UPDATE Integer SET _val = CASE _key " + + "WHEN 1 THEN 10 WHEN 2 THEN 20 ELSE 30 END")).getAll(); + + assertEquals(4L, res.get(0).get(0)); + + tx.rollback(); + } + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + res = cache.query(new SqlFieldsQuery("UPDATE Integer SET _val = CASE _val " + + "WHEN 1 THEN 10 WHEN 2 THEN 20 ELSE 30 END")).getAll(); + + assertEquals(4L, res.get(0).get(0)); + + res = cache.query(new SqlFieldsQuery("UPDATE Integer SET _val = CASE _val " + + "WHEN 10 THEN 100 WHEN 20 THEN 200 ELSE 300 END")).getAll(); + + assertEquals(4L, res.get(0).get(0)); + + res = cache.query(new SqlFieldsQuery("DELETE FROM Integer WHERE _key = 4")).getAll(); + + assertEquals(1L, res.get(0).get(0)); + + tx.commit(); + } + + IgniteCache cache0 = client.cache(DEFAULT_CACHE_NAME); + + Iterator> it = cache0.iterator(); + + Map map = new HashMap<>(); + + while (it.hasNext()) { + Cache.Entry e = it.next(); + + assertNull("duplicate key returned from iterator", map.putIfAbsent(e.getKey(), e.getValue())); + } + + assertEquals(3, map.size()); + + assertEquals(100, map.get(1).intValue()); + assertEquals(200, map.get(2).intValue()); + assertEquals(300, map.get(3).intValue()); + } + + /** + * @throws Exception If failed. + */ + public void testHints() throws Exception { + persistence = true; + + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + Ignite node = startGrid(getConfiguration("grid").setMvccVacuumTimeInterval(100)); + + node.cluster().active(true); + + Ignite client = startGrid(getConfiguration("client").setClientMode(true)); + + IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); + + List> res; + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + res = cache.query(new SqlFieldsQuery("INSERT INTO Integer (_key, _val) " + + "VALUES (1, 1), (2, 2), (3, 3), (4, 4)")).getAll(); + + assertEquals(4L, res.get(0).get(0)); + + tx.commit(); + } + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + res = cache.query(new SqlFieldsQuery("UPDATE Integer SET _val = CASE _key " + + "WHEN 1 THEN 10 WHEN 2 THEN 20 ELSE 30 END")).getAll(); + + assertEquals(4L, res.get(0).get(0)); + + tx.rollback(); + } + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + res = cache.query(new SqlFieldsQuery("UPDATE Integer SET _val = CASE _val " + + "WHEN 1 THEN 10 WHEN 2 THEN 20 ELSE 30 END")).getAll(); + + assertEquals(4L, res.get(0).get(0)); + + res = cache.query(new SqlFieldsQuery("UPDATE Integer SET _val = CASE _val " + + "WHEN 10 THEN 100 WHEN 20 THEN 200 ELSE 300 END")).getAll(); + + assertEquals(4L, res.get(0).get(0)); + + res = cache.query(new SqlFieldsQuery("DELETE FROM Integer WHERE _key = 4")).getAll(); + + assertEquals(1L, res.get(0).get(0)); + + tx.commit(); + } + + mvccProcessor(node).runVacuum().get(TX_TIMEOUT); + + checkAllVersionsHints(node.cache(DEFAULT_CACHE_NAME)); + } + + /** */ + private void checkAllVersionsHints(IgniteCache cache) throws IgniteCheckedException { + IgniteCacheProxy cache0 = (IgniteCacheProxy)cache; + GridCacheContext cctx = cache0.context(); + + assert cctx.mvccEnabled(); + + for (Object e : cache) { + IgniteBiTuple entry = (IgniteBiTuple)e; + + KeyCacheObject key = cctx.toCacheKeyObject(entry.getKey()); + + GridCursor cur = cctx.offheap().mvccAllVersionsCursor(cctx, key, CacheDataRowAdapter.RowData.LINK_WITH_HEADER); + + while (cur.next()) { + CacheDataRow row = cur.get(); + + assertTrue(row.mvccTxState() != 0); + } + } + } + + /** + * @param ex Exception holder. + * @param e Exception. + */ + private void onException(AtomicReference ex, T e) { + if (!ex.compareAndSet(null, e)) + ex.get().addSuppressed(e); + } + + /** + * @param phaser Phaser. + * @param phase Phase to wait for. + */ + private void awaitPhase(Phaser phaser, int phase) { + int p; + do { + p = phaser.arriveAndAwaitAdvance(); + } + while (p < phase); + } + + /** + * + */ + static class MvccTestSqlIndexValue implements Serializable { + /** */ + @QuerySqlField(index = true) + private int idxVal1; + + /** + * @param idxVal1 Indexed value 1. + */ + MvccTestSqlIndexValue(int idxVal1) { + this.idxVal1 = idxVal1; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + MvccTestSqlIndexValue value = (MvccTestSqlIndexValue)o; + return idxVal1 == value.idxVal1; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(idxVal1); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MvccTestSqlIndexValue.class, this); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java new file mode 100644 index 0000000000000..ade37638b9ec2 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java @@ -0,0 +1,829 @@ +/* + * 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.mvcc; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.FieldsQueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.testframework.GridTestUtils.runMultiThreaded; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Tests for transactional SQL. + */ +public abstract class CacheMvccSqlTxQueriesWithReducerAbstractTest extends CacheMvccAbstractTest { + /** */ + private static final int TIMEOUT = 3000; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + ccfgs = null; + ccfg = null; + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerInsert() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = + checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), + 2,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), + 3,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3))); + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT DISTINCT _key + 3, idxVal1 + 3 FROM MvccTestSqlIndexValue"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + qry.setDistributedJoins(true); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(4), cache.get(4)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(5), cache.get(5)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(6), cache.get(6)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerInsertDuplicateKey() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = + checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), + 2,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), + 3,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3))); + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT DISTINCT _key, idxVal1 FROM MvccTestSqlIndexValue"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + qry.setDistributedJoins(true); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Object call() { + return cache0.query(qry); + } + }, IgniteSQLException.class, "Duplicate key"); + + tx.rollback(); + } + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerMerge() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = + checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), + 2,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), + 3,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3))); + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "MERGE INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT DISTINCT _key * 2, idxVal1 FROM MvccTestSqlIndexValue"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + qry.setDistributedJoins(true); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(4)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(6)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerMultiBatchPerNodeServer() throws Exception { + checkMultiBatchPerNode(false); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerMultiBatchPerNodeClient() throws Exception { + checkMultiBatchPerNode(true); + } + + /** + * @throws Exception If failed. + */ + private void checkMultiBatchPerNode(boolean client) throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + Ignite checkNode; + Ignite updateNode; + + Random rnd = ThreadLocalRandom.current(); + + if (client) { + startGridsMultiThreaded(3); + + updateNode = grid(rnd.nextInt(3)); + + this.client = true; + + checkNode = startGrid(4); + } + else { + startGridsMultiThreaded(4); + + checkNode = grid(rnd.nextInt(4)); + updateNode = grid(rnd.nextInt(4)); + } + + IgniteCache cache = + checkNode.cache(DEFAULT_CACHE_NAME); + + final int count = 6; + + Map vals = new HashMap<>(count); + + for (int idx = 1; idx <= count; ++idx) + vals.put(idx, new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(idx)); + + cache.putAll(vals); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT DISTINCT _key + 6, idxVal1 + 6 FROM MvccTestSqlIndexValue"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + qry.setDistributedJoins(true); + qry.setPageSize(1); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals((long)count, cur.iterator().next().get(0)); + } + + tx.commit(); + } + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerDelete() throws Exception { + ccfgs = new CacheConfiguration[] { + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setName("int") + .setIndexedTypes(Integer.class, Integer.class), + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, + CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class), + }; + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache("int"); + + cache.putAll(F.asMap(1, 1, 3, 3, 5, 5)); + + final int count = 6; + + Map vals = new HashMap<>(count); + + for (int idx = 1; idx <= count; ++idx) + vals.put(idx, new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(idx)); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + cache0.putAll(vals); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "DELETE FROM MvccTestSqlIndexValue t " + + "WHERE EXISTS (SELECT 1 FROM \"int\".Integer WHERE t._key = _key)"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerUpdate() throws Exception { + ccfgs = new CacheConfiguration[] { + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setName("int") + .setIndexedTypes(Integer.class, Integer.class), + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, + CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class), + }; + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache("int"); + + cache.putAll(F.asMap(1, 5, 3, 1, 5, 3)); + + final int count = 6; + + Map vals = new HashMap<>(count); + + for (int idx = 1; idx <= count; ++idx) + vals.put(idx, new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(idx)); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + cache0.putAll(vals); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "UPDATE MvccTestSqlIndexValue t SET idxVal1=" + + "(SELECT _val FROM \"int\".Integer WHERE t._key = _key)" + + " WHERE EXISTS (SELECT 1 FROM \"int\".Integer WHERE t._key = _key)"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.commit(); + } + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerImplicitTxInsert() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = + checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), + 2,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), + 3,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3))); + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + + String sqlText = "INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT DISTINCT _key + 3, idxVal1 + 3 FROM MvccTestSqlIndexValue"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + qry.setTimeout(TX_TIMEOUT, TimeUnit.MILLISECONDS); + + qry.setDistributedJoins(true); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(4), cache.get(4)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(5), cache.get(5)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(6), cache.get(6)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerRollbackInsert() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = + checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), + 2,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), + 3,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3))); + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + + try (Transaction tx = updateNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT DISTINCT _key + 3, idxVal1 + 3 FROM MvccTestSqlIndexValue"; + + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText); + + qry.setDistributedJoins(true); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + tx.rollback(); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), sqlGet(1, cache).get(0).get(0)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), sqlGet(2, cache).get(0).get(0)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), sqlGet(3, cache).get(0).get(0)); + assertTrue(sqlGet(4, cache).isEmpty()); + assertTrue(sqlGet(5, cache).isEmpty()); + assertTrue(sqlGet(6, cache).isEmpty()); + } + + /** + * @param key Key. + * @param cache Cache. + * @return Result. + */ + private List sqlGet(int key, IgniteCache cache) { + return cache.query(new SqlFieldsQuery("SELECT _val from MvccTestSqlIndexValue WHERE _key=" + key)).getAll(); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerDeadlockInsert() throws Exception { + ccfgs = new CacheConfiguration[] { + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setName("int") + .setIndexedTypes(Integer.class, Integer.class), + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, + CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class), + }; + + startGridsMultiThreaded(2); + + client = true; + + startGridsMultiThreaded(2, 2); + + Ignite checkNode = grid(2); + + IgniteCache cache = checkNode.cache("int"); + + HashMap vals = new HashMap<>(100); + + for (int idx = 0; idx < 100; ++idx) + vals.put(idx, idx); + + cache.putAll(vals); + + final CyclicBarrier barrier = new CyclicBarrier(2); + final AtomicInteger idx = new AtomicInteger(2); + final AtomicReference ex = new AtomicReference<>(); + + multithreaded(new Runnable() { + @Override public void run() { + int id = idx.getAndIncrement(); + + IgniteEx node = grid(id); + + try { + try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TIMEOUT); + + String sqlText = "INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT DISTINCT _key, _val FROM \"int\".Integer ORDER BY _key"; + + String sqlAsc = sqlText + " ASC"; + String sqlDesc = sqlText + " DESC"; + + SqlFieldsQuery qry = new SqlFieldsQuery((id % 2) == 0 ? sqlAsc : sqlDesc); + + IgniteCache cache0 = node.cache(DEFAULT_CACHE_NAME); + + cache0.query(qry).getAll(); + + barrier.await(); + + qry = new SqlFieldsQuery((id % 2) == 0 ? sqlDesc : sqlAsc); + + cache0.query(qry).getAll(); + + tx.commit(); + } + } + catch (Exception e) { + onException(ex, e); + } + } + }, 2); + + Exception ex0 = ex.get(); + + assertNotNull(ex0); + + if (!X.hasCause(ex0, IgniteTxTimeoutCheckedException.class)) + throw ex0; + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerInsertVersionConflict() throws Exception { + ccfgs = new CacheConfiguration[] { + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setName("int") + .setIndexedTypes(Integer.class, Integer.class), + cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, + CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class), + }; + + startGridsMultiThreaded(2); + + client = true; + + final Ignite checkNode = startGrid(2); + + IgniteCache cache = checkNode.cache("int"); + + HashMap vals = new HashMap<>(100); + + for (int idx = 0; idx < 10; ++idx) + vals.put(idx, idx); + + cache.putAll(vals); + + awaitPartitionMapExchange(); + + IgniteCache cache0 = checkNode.cache(DEFAULT_CACHE_NAME); + + cache0.query(new SqlFieldsQuery("INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) " + + "SELECT _key, _val FROM \"int\".Integer")).getAll(); + + final CyclicBarrier barrier = new CyclicBarrier(2); + final AtomicReference ex = new AtomicReference<>(); + + runMultiThreaded(new Runnable() { + @Override public void run() { + try { + try (Transaction tx = checkNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + tx.timeout(TX_TIMEOUT); + + barrier.await(); + + SqlFieldsQuery qry = new SqlFieldsQuery("SELECT * FROM MvccTestSqlIndexValue"); + + cache0.query(qry).getAll(); + + barrier.await(); + + String sqlText = "UPDATE MvccTestSqlIndexValue t SET idxVal1=" + + "(SELECT _val FROM \"int\".Integer WHERE t._key = _key ORDER BY _key)"; + + qry = new SqlFieldsQuery(sqlText); + + cache0.query(qry).getAll(); + + tx.commit(); + } + } + catch (Exception e) { + onException(ex, e); + } + } + }, 2, "tx-thread"); + + IgniteSQLException ex0 = X.cause(ex.get(), IgniteSQLException.class); + + assertNotNull("Exception has not been thrown.", ex0); + assertEquals("Mvcc version mismatch.", ex0.getMessage()); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerInsertValues() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite node = grid(rnd.nextInt(4)); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO MvccTestSqlIndexValue (_key, idxVal1)" + + " values (1,?),(2,?),(3,?)"); + + qry.setArgs(1, 2, 3); + + try (FieldsQueryCursor> cur = cache.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + + qry = new SqlFieldsQuery("INSERT INTO MvccTestSqlIndexValue (_key, idxVal1) values (4,4)"); + + try (FieldsQueryCursor> cur = cache.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(4), cache.get(4)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerMergeValues() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite node = grid(rnd.nextInt(4)); + + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + cache.put(1, new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1)); + cache.put(3, new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3)); + + SqlFieldsQuery qry = new SqlFieldsQuery("MERGE INTO MvccTestSqlIndexValue (_key, idxVal1)" + + " values (1,?),(2,?),(3,?)"); + + qry.setArgs(1, 4, 6); + + try (FieldsQueryCursor> cur = cache.query(qry)) { + assertEquals(3L, cur.iterator().next().get(0)); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(4), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(6), cache.get(3)); + + qry = new SqlFieldsQuery("MERGE INTO MvccTestSqlIndexValue (_key, idxVal1) values (4,4)"); + + try (FieldsQueryCursor> cur = cache.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(4), cache.get(4)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerFastUpdate() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, Integer.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap(1,1,2,2,3,3)); + + assertEquals(1, cache.get(1)); + assertEquals(2, cache.get(2)); + assertEquals(3, cache.get(3)); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("UPDATE Integer SET _val = 8 WHERE _key = ?").setArgs(1); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + qry = new SqlFieldsQuery("UPDATE Integer SET _val = 9 WHERE _key = 2"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + assertEquals(8, cache.get(1)); + assertEquals(9, cache.get(2)); + assertEquals(3, cache.get(3)); + } + + /** + * @throws Exception If failed. + */ + public void testQueryReducerFastDelete() throws Exception { + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + .setIndexedTypes(Integer.class, CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue.class); + + startGridsMultiThreaded(4); + + Random rnd = ThreadLocalRandom.current(); + + Ignite checkNode = grid(rnd.nextInt(4)); + Ignite updateNode = grid(rnd.nextInt(4)); + + IgniteCache cache = checkNode.cache(DEFAULT_CACHE_NAME); + + cache.putAll(F.asMap( + 1,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), + 2,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), + 3,new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3))); + + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(1), cache.get(1)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(2), cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + + IgniteCache cache0 = updateNode.cache(DEFAULT_CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("DELETE FROM MvccTestSqlIndexValue WHERE _key = ?") + .setArgs(1); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + qry = new SqlFieldsQuery("DELETE FROM MvccTestSqlIndexValue WHERE _key = 2"); + + try (FieldsQueryCursor> cur = cache0.query(qry)) { + assertEquals(1L, cur.iterator().next().get(0)); + } + + assertNull(cache.get(1)); + assertNull(cache.get(2)); + assertEquals(new CacheMvccSqlTxQueriesAbstractTest.MvccTestSqlIndexValue(3), cache.get(3)); + } + + /** + * @param ex Exception holder. + * @param e Exception. + */ + private void onException(AtomicReference ex, Exception e) { + if (!ex.compareAndSet(null, e)) + ex.get().addSuppressed(e); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccStreamingInsertTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccStreamingInsertTest.java new file mode 100644 index 0000000000000..7d2e335c416b2 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccStreamingInsertTest.java @@ -0,0 +1,122 @@ +/* + * 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.mvcc; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteJdbcDriver; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; + +import static java.util.Arrays.asList; + +/** + * + */ +public class CacheMvccStreamingInsertTest extends CacheMvccAbstractTest { + /** */ + private IgniteCache sqlNexus; + + /** */ + private Connection conn; + + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + Ignite ignite = startGrid(0); + sqlNexus = ignite.getOrCreateCache(new CacheConfiguration<>("sqlNexus").setSqlSchema("PUBLIC")); + sqlNexus.query(q("" + + "create table person(" + + " id int not null primary key," + + " name varchar not null" + + ") with \"atomicity=transactional\"" + )); + + Properties props = new Properties(); + props.setProperty(IgniteJdbcDriver.PROP_STREAMING, "true"); + conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1", props); + } + + /** + * @throws Exception If failed. + */ + public void testStreamingInsertWithoutOverwrite() throws Exception { + conn.createStatement().execute("SET STREAMING 1 BATCH_SIZE 2 ALLOW_OVERWRITE 0 " + + " PER_NODE_BUFFER_SIZE 1000 FLUSH_FREQUENCY 100"); + sqlNexus.query(q("insert into person values(1, 'ivan')")); + + PreparedStatement batchStmt = conn.prepareStatement("insert into person values(?, ?)"); + batchStmt.setInt(1, 1); + batchStmt.setString(2, "foo"); + batchStmt.addBatch(); + batchStmt.setInt(1, 2); + batchStmt.setString(2, "bar"); + batchStmt.addBatch(); + TimeUnit.MILLISECONDS.sleep(500); + + List> rows = sqlNexus.query(q("select * from person")).getAll(); + List> exp = asList( + asList(1, "ivan"), + asList(2, "bar") + ); + assertEquals(exp, rows); + } + + /** + * @throws Exception If failed. + */ + public void testUpdateWithOverwrite() throws Exception { + conn.createStatement().execute("SET STREAMING 1 BATCH_SIZE 2 ALLOW_OVERWRITE 1 " + + " PER_NODE_BUFFER_SIZE 1000 FLUSH_FREQUENCY 100"); + sqlNexus.query(q("insert into person values(1, 'ivan')")); + + PreparedStatement batchStmt = conn.prepareStatement("insert into person values(?, ?)"); + batchStmt.setInt(1, 1); + batchStmt.setString(2, "foo"); + batchStmt.addBatch(); + batchStmt.setInt(1, 2); + batchStmt.setString(2, "bar"); + batchStmt.addBatch(); + TimeUnit.MILLISECONDS.sleep(500); + + List> rows = sqlNexus.query(q("select * from person")).getAll(); + List> exp = asList( + asList(1, "foo"), + asList(2, "bar") + ); + assertEquals(exp, rows); + } + + /** */ + private static SqlFieldsQuery q(String sql) { + return new SqlFieldsQuery(sql); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java new file mode 100644 index 0000000000000..57cee61ecd209 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java @@ -0,0 +1,350 @@ +/* + * 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.File; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.index.DynamicIndexAbstractSelfTest; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; +import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl; +import org.apache.ignite.internal.util.lang.GridCursor; +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; +import org.apache.ignite.lang.IgniteBiTuple; + +/** + * Index rebuild after node restart test. + */ +public class GridIndexRebuildSelfTest extends DynamicIndexAbstractSelfTest { + /** Data size. */ + private final static int AMOUNT = 10; + + /** Data size. */ + private final static String CACHE_NAME = "T"; + + /** Test instance to allow interaction with static context. */ + private static GridIndexRebuildSelfTest INSTANCE; + + /** Latch to signal that rebuild may start. */ + private final CountDownLatch rebuildLatch = new CountDownLatch(1); + + /** Latch to signal that concurrent put may start. */ + private final Semaphore rebuildSemaphore = new Semaphore(1, true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration commonConfiguration(int idx) throws Exception { + IgniteConfiguration cfg = super.commonConfiguration(idx); + + cfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration().setPersistenceEnabled(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + // Just in case. + cleanPersistenceDir(); + + INSTANCE = this; + } + + /** + * Do test with MVCC enabled. + */ + public void testMvccEnabled() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-7259"); + doTest(true); + } + + /** + * Do test with MVCC disabled. + */ + public void testMvccDisabled() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-7259"); + doTest(false); + } + + /** + * Do test.

    + * Steps are as follows: + *

      + *
    • Put some data;
    • + *
    • Stop the node;
    • + *
    • Remove index file;
    • + *
    • Restart the node and block index rebuild;
    • + *
    • For half of the keys do cache puts before corresponding key + * has been processed during index rebuild;
    • + *
    • Check that: + *
        + *
      • For MVCC case: some keys have all versions that existed before restart, while those + * updated concurrently have only put version (one with mark value -1) + * and latest version present before node restart;
      • + *
      • For non MVCC case: keys updated concurrently must have mark values of -1 despite that + * index rebuild for them has happened after put.
      • + *
      + *
    • + *
    + * @param mvccEnabled MVCC flag. + * @throws Exception if failed. + */ + private void doTest(boolean mvccEnabled) throws Exception { + IgniteEx srv = startServer(mvccEnabled); + + execute(srv, "CREATE TABLE T(k int primary key, v int) WITH \"cache_name=T,wrap_value=false," + + "atomicity=transactional\""); + + execute(srv, "CREATE INDEX IDX ON T(v)"); + + IgniteInternalCache cc = srv.cachex(CACHE_NAME); + + assertNotNull(cc); + + if (mvccEnabled) + lockVersion(srv); + + putData(srv, false); + + checkDataState(srv, mvccEnabled, false); + + File cacheWorkDir = ((FilePageStoreManager)cc.context().shared().pageStore()).cacheWorkDir(cc.configuration()); + + File idxPath = cacheWorkDir.toPath().resolve("index.bin").toFile(); + + stopAllGrids(); + + assertTrue(U.delete(idxPath)); + + srv = startServer(mvccEnabled); + + putData(srv, true); + + checkDataState(srv, mvccEnabled, true); + } + + /** + * Check versions presence in index tree. + * @param srv Node. + * @param mvccEnabled MVCC flag. + * @param afterRebuild Whether index rebuild has occurred. + * @throws IgniteCheckedException if failed. + */ + @SuppressWarnings({"ConstantConditions", "unchecked"}) + private void checkDataState(IgniteEx srv, boolean mvccEnabled, boolean afterRebuild) throws IgniteCheckedException { + IgniteInternalCache icache = srv.cachex(CACHE_NAME); + + IgniteCache cache = srv.cache(CACHE_NAME); + + assertNotNull(icache); + + for (IgniteCacheOffheapManager.CacheDataStore store : icache.context().offheap().cacheDataStores()) { + GridCursor cur = store.cursor(); + + while (cur.next()) { + CacheDataRow row = cur.get(); + + int key = row.key().value(icache.context().cacheObjectContext(), false); + + if (mvccEnabled) { + List> vers = store.mvccFindAllVersions(icache.context(), row.key()); + + if (!afterRebuild || key <= AMOUNT / 2) + assertEquals(key, vers.size()); + else { + // For keys affected by concurrent put there are two versions - + // -1 (concurrent put mark) and newest restored value as long as put cleans obsolete versions. + assertEquals(2, vers.size()); + + assertEquals(-1, vers.get(0).getKey()); + assertEquals(key, vers.get(1).getKey()); + } + } + else { + if (!afterRebuild || key <= AMOUNT / 2) + assertEquals(key, cache.get(key)); + else + assertEquals(-1, cache.get(key)); + } + } + } + } + + /** + * Lock coordinator version in order to keep MVCC versions in place. + * @param node Node. + * @throws IgniteCheckedException if failed. + */ + private static void lockVersion(IgniteEx node) throws IgniteCheckedException { + node.context().coordinators().requestSnapshotAsync().get(); + } + + /** + * Put data to cache. + * @param node Node. + * @throws Exception if failed. + */ + private void putData(Ignite node, final boolean forConcurrentPut) throws Exception { + final IgniteCache cache = node.cache(CACHE_NAME); + + assertNotNull(cache); + + for (int i = 1; i <= AMOUNT; i++) { + if (forConcurrentPut) { + // Concurrent put affects only second half of the keys. + if (i <= AMOUNT / 2) + continue; + + rebuildSemaphore.acquire(); + + cache.put(i, -1); + + rebuildLatch.countDown(); + + rebuildSemaphore.release(); + } + else { + // Data streamer is not used intentionally in order to preserve all versions. + for (int j = 1; j <= i; j++) + cache.put(i, j); + } + } + } + + /** + * Start server node. + * @param mvccEnabled MVCC flag. + * @return Started node. + * @throws Exception if failed. + */ + private IgniteEx startServer(boolean mvccEnabled) throws Exception { + // Have to do this for each starting node - see GridQueryProcessor ctor, it nulls + // idxCls static field on each call. + GridQueryProcessor.idxCls = BlockingIndexing.class; + + IgniteConfiguration cfg = serverConfiguration(0).setMvccEnabled(mvccEnabled); + + IgniteEx res = startGrid(cfg); + + res.active(true); + + return res; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + cleanPersistenceDir(); + } + + /** + * Blocking indexing processor. + */ + private static class BlockingIndexing extends IgniteH2Indexing { + /** Flag to ignore first rebuild performed on initial node start. */ + private boolean firstRbld = true; + + /** {@inheritDoc} */ + @Override public void rebuildIndexesFromHash(String cacheName) throws IgniteCheckedException { + if (!firstRbld) + U.await(INSTANCE.rebuildLatch); + else + firstRbld = false; + + int cacheId = CU.cacheId(cacheName); + + GridCacheContext cctx = ctx.cache().context().cacheContext(cacheId); + + final GridCacheQueryManager qryMgr = cctx.queries(); + + SchemaIndexCacheVisitor visitor = new SchemaIndexCacheVisitorImpl(cctx); + + visitor.visit(new TestRebuildClosure(qryMgr, cctx.mvccEnabled())); + + for (H2TableDescriptor tblDesc : tables(cacheName)) + tblDesc.table().markRebuildFromHashInProgress(false); + } + } + + /** + * Test closure. + */ + private final static class TestRebuildClosure extends RebuildIndexFromHashClosure { + /** Seen keys set to track moment when concurrent put may start. */ + private final Set keys = + Collections.newSetFromMap(new ConcurrentHashMap()); + + /** + * @param qryMgr Query manager. + * @param mvccEnabled MVCC status flag. + */ + TestRebuildClosure(GridCacheQueryManager qryMgr, boolean mvccEnabled) { + super(qryMgr, mvccEnabled); + } + + /** {@inheritDoc} */ + @Override public synchronized void apply(CacheDataRow row) throws IgniteCheckedException { + // For half of the keys, we want to do rebuild + // after corresponding key had been put from a concurrent thread. + boolean keyFirstMet = keys.add(row.key()) && keys.size() > AMOUNT / 2; + + if (keyFirstMet) { + try { + INSTANCE.rebuildSemaphore.acquire(); + } + catch (InterruptedException e) { + throw new IgniteCheckedException(e); + } + } + + super.apply(row); + + if (keyFirstMet) + INSTANCE.rebuildSemaphore.release(); + } + } +} 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 235b28b33778c..6b76230590e85 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 @@ -343,7 +343,7 @@ public void testSpi() throws Exception { // Fields query GridQueryFieldsResult fieldsRes = spi.queryLocalSqlFields(spi.schema("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, false, 0, null); String[] aliases = {"N1", "A1", "N2", "A2"}; Object[] vals = { "Valera", 19, "Kolya", 25}; @@ -401,7 +401,7 @@ public void testLongQueries() throws Exception { range *= 3; GridQueryFieldsResult res = spi.queryLocalSqlFields(spi.schema("A"), sql, Arrays.asList(1, - range), null, false, 0, null); + range), null, false, false, 0, null); assert res.iterator().hasNext(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2StatementCacheSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2StatementCacheSelfTest.java new file mode 100644 index 0000000000000..655d039632386 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/H2StatementCacheSelfTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2; + +import java.sql.PreparedStatement; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class H2StatementCacheSelfTest extends GridCommonAbstractTest { + + /** + * @throws Exception If failed. + */ + public void testEviction() throws Exception { + H2StatementCache stmtCache = new H2StatementCache(1); + H2CachedStatementKey key1 = new H2CachedStatementKey("", "1"); + PreparedStatement stmt1 = stmt(); + stmtCache.put(key1, stmt1); + + assertSame(stmt1, stmtCache.get(key1)); + + stmtCache.put(new H2CachedStatementKey("mydb", "2"), stmt()); + + assertNull(stmtCache.get(key1)); + } + + /** + * @throws Exception If failed. + */ + public void testLruEvictionInStoreOrder() throws Exception { + H2StatementCache stmtCache = new H2StatementCache(2); + + H2CachedStatementKey key1 = new H2CachedStatementKey("", "1"); + H2CachedStatementKey key2 = new H2CachedStatementKey("", "2"); + stmtCache.put(key1, stmt()); + stmtCache.put(key2, stmt()); + + stmtCache.put(new H2CachedStatementKey("", "3"), stmt()); + + assertNull(stmtCache.get(key1)); + } + + /** + * @throws Exception If failed. + */ + public void testLruEvictionInAccessOrder() throws Exception { + H2StatementCache stmtCache = new H2StatementCache(2); + + H2CachedStatementKey key1 = new H2CachedStatementKey("", "1"); + H2CachedStatementKey key2 = new H2CachedStatementKey("", "2"); + stmtCache.put(key1, stmt()); + stmtCache.put(key2, stmt()); + stmtCache.get(key1); + + stmtCache.put(new H2CachedStatementKey("", "3"), stmt()); + + assertNull(stmtCache.get(key2)); + } + + /** + * + */ + private static PreparedStatement stmt() { + return new PreparedStatementExImpl(null); + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExSelfTest.java new file mode 100644 index 0000000000000..22bff3b2a9b9b --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/PreparedStatementExSelfTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2; + +import java.sql.PreparedStatement; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class PreparedStatementExSelfTest extends GridCommonAbstractTest { + /** + * @throws Exception If failed. + */ + public void testStoringMeta() throws Exception { + PreparedStatement stmt = stmt(); + + PreparedStatementEx wrapped = stmt.unwrap(PreparedStatementEx.class); + + wrapped.putMeta(0, "0"); + + assertEquals("0", wrapped.meta(0)); + } + + /** + * @throws Exception If failed. + */ + public void testStoringMoreMetaKeepsExisting() throws Exception { + PreparedStatement stmt = stmt(); + + PreparedStatementEx wrapped = stmt.unwrap(PreparedStatementEx.class); + + wrapped.putMeta(0, "0"); + wrapped.putMeta(1, "1"); + + assertEquals("0", wrapped.meta(0)); + assertEquals("1", wrapped.meta(1)); + } + + /** + * + */ + private static PreparedStatement stmt() { + return new PreparedStatementExImpl(null); + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java new file mode 100644 index 0000000000000..b7b7a3701c975 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java @@ -0,0 +1,113 @@ +/* + * 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.util.concurrent.CompletableFuture; +import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPool.Reusable; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class ThreadLocalObjectPoolSelfTest extends GridCommonAbstractTest { + /** */ + private ThreadLocalObjectPool pool = new ThreadLocalObjectPool<>(Obj::new, 1); + + /** + * @throws Exception If failed. + */ + public void testObjectIsReusedAfterRecycling() throws Exception { + Reusable o1 = pool.borrow(); + o1.recycle(); + Reusable o2 = pool.borrow(); + + assertSame(o1.object(), o2.object()); + assertFalse(o1.object().isClosed()); + } + + /** + * @throws Exception If failed. + */ + public void testBorrowedObjectIsNotReturnedTwice() throws Exception { + Reusable o1 = pool.borrow(); + Reusable o2 = pool.borrow(); + + assertNotSame(o1.object(), o2.object()); + } + + /** + * @throws Exception If failed. + */ + public void testObjectShouldBeClosedOnRecycleIfPoolIsFull() throws Exception { + Reusable o1 = pool.borrow(); + Reusable o2 = pool.borrow(); + o1.recycle(); + o2.recycle(); + + assertTrue(o2.object().isClosed()); + } + + /** + * @throws Exception If failed. + */ + public void testObjectShouldNotBeReturnedIfPoolIsFull() throws Exception { + Reusable o1 = pool.borrow(); + Reusable o2 = pool.borrow(); + + o1.recycle(); + + assertEquals(1, pool.bagSize()); + + o2.recycle(); + + assertEquals(1, pool.bagSize()); + } + + /** + * @throws Exception If failed. + */ + public void testObjectShouldReturnedToRecyclingThreadBag() throws Exception { + Reusable o1 = pool.borrow(); + + CompletableFuture.runAsync(() -> { + o1.recycle(); + + assertEquals(1, pool.bagSize()); + }).join(); + + assertEquals(0, pool.bagSize()); + } + + /** */ + private static class Obj implements AutoCloseable { + /** */ + private boolean closed = false; + + /** {@inheritDoc} */ + @Override public void close() { + closed = true; + } + + /** + * @return {@code True} if closed. + */ + public boolean isClosed() { + return closed; + } + } +} 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 a362586a0a36d..de77150c51d34 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 @@ -205,6 +205,7 @@ public void testParseSelectAndUnion() throws Exception { checkQuery("select * from Person"); checkQuery("select distinct * from Person"); checkQuery("select p.name, date from Person p"); + checkQuery("select p.name, date from Person p for update"); checkQuery("select * from Person p, sch2.Address a"); checkQuery("select * from Person, sch2.Address"); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java new file mode 100644 index 0000000000000..a8087da077850 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java @@ -0,0 +1,73 @@ +/* + * 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.cache.mvcc.CacheMvccBulkLoadTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccDmlSimpleTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccIteratorWithConcurrentJdbcTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedBackupsTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedSelectForUpdateQueryTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedSqlCoordinatorFailoverTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedSqlQueriesTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedSqlTxQueriesTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccPartitionedSqlTxQueriesWithReducerTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedBackupsTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSelectForUpdateQueryTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSqlCoordinatorFailoverTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSqlQueriesTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSqlTxQueriesTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSqlTxQueriesWithReducerTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccScanQueryWithConcurrentJdbcTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentJdbcTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccStreamingInsertTest; + +/** + * + */ +public class IgniteCacheMvccSqlTestSuite extends TestSuite { + /** + * @return Test suite. + */ + public static TestSuite suite() { + TestSuite suite = new TestSuite("IgniteCache SQL MVCC Test Suite"); + + suite.addTestSuite(CacheMvccSizeWithConcurrentJdbcTransactionTest.class); + suite.addTestSuite(CacheMvccScanQueryWithConcurrentJdbcTransactionTest.class); + suite.addTestSuite(CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest.class); + suite.addTestSuite(CacheMvccIteratorWithConcurrentJdbcTransactionTest.class); + suite.addTestSuite(CacheMvccPartitionedSqlQueriesTest.class); + suite.addTestSuite(CacheMvccReplicatedSqlQueriesTest.class); + suite.addTestSuite(CacheMvccPartitionedSqlTxQueriesTest.class); + suite.addTestSuite(CacheMvccReplicatedSqlTxQueriesTest.class); + suite.addTestSuite(CacheMvccPartitionedSqlTxQueriesWithReducerTest.class); + suite.addTestSuite(CacheMvccReplicatedSqlTxQueriesWithReducerTest.class); + suite.addTestSuite(CacheMvccPartitionedSelectForUpdateQueryTest.class); + suite.addTestSuite(CacheMvccReplicatedSelectForUpdateQueryTest.class); + suite.addTestSuite(CacheMvccPartitionedBackupsTest.class); + suite.addTestSuite(CacheMvccReplicatedBackupsTest.class); + suite.addTestSuite(CacheMvccPartitionedSqlCoordinatorFailoverTest.class); + suite.addTestSuite(CacheMvccReplicatedSqlCoordinatorFailoverTest.class); + suite.addTestSuite(CacheMvccBulkLoadTest.class); + suite.addTestSuite(CacheMvccStreamingInsertTest.class); + suite.addTestSuite(CacheMvccDmlSimpleTest.class); + + return suite; + } +} 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 2a61ce6115028..b0670e80d3187 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.CacheReplicatedQueryMetricsDistributedSelfTest; import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest; +import org.apache.ignite.internal.processors.cache.DdlTransactionSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest; import org.apache.ignite.internal.processors.cache.GridCacheFullTextQuerySelfTest; import org.apache.ignite.internal.processors.cache.GridCacheLazyQueryPartitionsReleaseTest; @@ -133,6 +134,8 @@ import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest; import org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest; import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest; +import org.apache.ignite.internal.processors.cache.index.SqlTransactionsComandsWithMvccDisabledSelfTest; +import org.apache.ignite.internal.processors.cache.index.SqlTransactionsSelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest; @@ -167,9 +170,13 @@ import org.apache.ignite.internal.processors.query.SqlSchemaSelfTest; import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest; import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest; +import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildSelfTest; +import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest; import org.apache.ignite.internal.processors.query.h2.H2ResultSetIteratorNullifyOnEndSelfTest; import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest; import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest; +import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest; +import org.apache.ignite.internal.processors.query.h2.PreparedStatementExSelfTest; import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest; import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest; import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest; @@ -184,6 +191,7 @@ import org.apache.ignite.internal.sql.SqlParserCreateIndexSelfTest; import org.apache.ignite.internal.sql.SqlParserDropIndexSelfTest; import org.apache.ignite.internal.sql.SqlParserSetStreamingSelfTest; +import org.apache.ignite.internal.sql.SqlParserTransactionalKeywordsSelfTest; import org.apache.ignite.internal.sql.SqlParserUserSelfTest; import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest; import org.apache.ignite.sqltests.PartitionedSqlTest; @@ -206,6 +214,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(SqlParserCreateIndexSelfTest.class); suite.addTestSuite(SqlParserDropIndexSelfTest.class); + suite.addTestSuite(SqlParserTransactionalKeywordsSelfTest.class); suite.addTestSuite(SqlParserBulkLoadSelfTest.class); suite.addTestSuite(SqlParserSetStreamingSelfTest.class); @@ -354,6 +363,8 @@ public static TestSuite suite() throws Exception { //suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.class); suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalReplicatedTest.class); + suite.addTestSuite(DdlTransactionSelfTest.class); + // Fields queries. suite.addTestSuite(SqlFieldsQuerySelfTest.class); suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class); @@ -434,6 +445,11 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(OptimizedMarshallerIndexNameTest.class); suite.addTestSuite(SqlSystemViewsSelfTest.class); + suite.addTestSuite(GridIndexRebuildSelfTest.class); + + suite.addTestSuite(SqlTransactionsSelfTest.class); + suite.addTestSuite(SqlTransactionsComandsWithMvccDisabledSelfTest.class); + suite.addTestSuite(IgniteSqlDefaultValueTest.class); suite.addTestSuite(IgniteDecimalSelfTest.class); suite.addTestSuite(IgniteSQLColumnConstraintsTest.class); @@ -451,6 +467,10 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(SqlParserUserSelfTest.class); suite.addTestSuite(SqlUserCommandSelfTest.class); + suite.addTestSuite(ThreadLocalObjectPoolSelfTest.class); + suite.addTestSuite(H2StatementCacheSelfTest.class); + suite.addTestSuite(PreparedStatementExSelfTest.class); + return suite; } } diff --git a/modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person.csv b/modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person.csv new file mode 100644 index 0000000000000..ef7a087f839ae --- /dev/null +++ b/modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person.csv @@ -0,0 +1,2 @@ +1,John +2,Jack diff --git a/modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person_broken.csv b/modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person_broken.csv new file mode 100644 index 0000000000000..b5c2b3fed757b --- /dev/null +++ b/modules/indexing/src/test/resources/org/apache/ignite/internal/processors/cache/mvcc/mvcc_person_broken.csv @@ -0,0 +1,2 @@ +1,John +2 diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java index 8dcdd574fa6b8..88c86b2040aca 100644 --- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java +++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java @@ -167,6 +167,7 @@ private CacheTmLookup createTmLookup(String tmLookupClsName) throws IgniteChecke tCfg.getDefaultTxIsolation(), tCfg.getDefaultTxTimeout(), /*store enabled*/true, + /*sql*/false, /*tx size*/0, null ); diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index f41240365b30d..14dae7cd7d322 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -85,6 +85,7 @@ ignite_odbc_tests_SOURCES = \ src/errors_test.cpp \ src/odbc_test_suite.cpp \ src/types_test.cpp \ + src/transaction_test.cpp \ src/authentication_test.cpp \ ../odbc/src/log.cpp \ ../odbc/src/cursor.cpp \ @@ -101,7 +102,8 @@ ignite_odbc_tests_SOURCES = \ ../odbc/src/column.cpp \ ../odbc/src/common_types.cpp \ ../odbc/src/utility.cpp \ - ../odbc/src/result_page.cpp + ../odbc/src/result_page.cpp \ + ../odbc/src/nested_tx_mode.cpp run-check: check ./ignite-odbc-tests -p diff --git a/modules/platforms/cpp/odbc-test/config/queries-transaction-32.xml b/modules/platforms/cpp/odbc-test/config/queries-transaction-32.xml new file mode 100644 index 0000000000000..94ddd3d81eab8 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/config/queries-transaction-32.xml @@ -0,0 +1,50 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/platforms/cpp/odbc-test/config/queries-transaction.xml b/modules/platforms/cpp/odbc-test/config/queries-transaction.xml new file mode 100644 index 0000000000000..7d74fc9731063 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/config/queries-transaction.xml @@ -0,0 +1,32 @@ + + + + + + + + + + + + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index 87b655936c1f5..3410ec5fcc626 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -164,6 +164,7 @@ + @@ -200,6 +201,7 @@ + @@ -233,6 +235,8 @@ + + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 3d2fcc6dfd13c..3065df031353b 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -145,6 +145,9 @@ Code + + Code + Externals @@ -166,6 +169,9 @@ Code + + Externals + @@ -209,5 +215,11 @@ Configs + + Configs + + + Configs + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp index 630597c2e64db..784270410509c 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_get_info_test.cpp @@ -136,7 +136,7 @@ BOOST_AUTO_TEST_CASE(TestValues) CheckStrInfo(SQL_LIKE_ESCAPE_CLAUSE, "N"); CheckStrInfo(SQL_MAX_ROW_SIZE_INCLUDES_LONG, "Y"); CheckStrInfo(SQL_MULT_RESULT_SETS, "N"); - CheckStrInfo(SQL_MULTIPLE_ACTIVE_TXN, "N"); + CheckStrInfo(SQL_MULTIPLE_ACTIVE_TXN, "Y"); CheckStrInfo(SQL_ORDER_BY_COLUMNS_IN_SELECT, "N"); CheckStrInfo(SQL_PROCEDURE_TERM, "stored procedure"); CheckStrInfo(SQL_PROCEDURES, "N"); @@ -182,7 +182,7 @@ BOOST_AUTO_TEST_CASE(TestValues) CheckIntInfo(SQL_CREATE_VIEW, 0); CheckIntInfo(SQL_CURSOR_SENSITIVITY, SQL_INSENSITIVE); CheckIntInfo(SQL_DDL_INDEX, SQL_DI_CREATE_INDEX | SQL_DI_DROP_INDEX); - CheckIntInfo(SQL_DEFAULT_TXN_ISOLATION, 0); + CheckIntInfo(SQL_DEFAULT_TXN_ISOLATION, SQL_TXN_REPEATABLE_READ); CheckIntInfo(SQL_DROP_ASSERTION, 0); CheckIntInfo(SQL_DROP_CHARACTER_SET, 0); CheckIntInfo(SQL_DROP_COLLATION, 0); @@ -212,7 +212,7 @@ BOOST_AUTO_TEST_CASE(TestValues) CheckIntInfo(SQL_SQL92_GRANT, 0); CheckIntInfo(SQL_SQL92_REVOKE, 0); CheckIntInfo(SQL_STANDARD_CLI_CONFORMANCE, 0); - CheckIntInfo(SQL_TXN_ISOLATION_OPTION, 0); + CheckIntInfo(SQL_TXN_ISOLATION_OPTION, SQL_TXN_REPEATABLE_READ); CheckIntInfo(SQL_UNION, SQL_U_UNION | SQL_U_UNION_ALL); CheckIntInfo(SQL_SCHEMA_USAGE, SQL_SU_DML_STATEMENTS | SQL_SU_TABLE_DEFINITION | SQL_SU_PRIVILEGE_DEFINITION | @@ -384,7 +384,7 @@ BOOST_AUTO_TEST_CASE(TestValues) CheckShortInfo(SQL_MAX_CONCURRENT_ACTIVITIES, 0); CheckShortInfo(SQL_CURSOR_COMMIT_BEHAVIOR, SQL_CB_PRESERVE); CheckShortInfo(SQL_CURSOR_ROLLBACK_BEHAVIOR, SQL_CB_PRESERVE); - CheckShortInfo(SQL_TXN_CAPABLE, SQL_TC_NONE); + CheckShortInfo(SQL_TXN_CAPABLE, SQL_TC_DDL_COMMIT); CheckShortInfo(SQL_QUOTED_IDENTIFIER_CASE, SQL_IC_SENSITIVE); CheckShortInfo(SQL_ACTIVE_ENVIRONMENTS, 0); CheckShortInfo(SQL_CONCAT_NULL_BEHAVIOR, SQL_CB_NULL); diff --git a/modules/platforms/cpp/odbc-test/src/transaction_test.cpp b/modules/platforms/cpp/odbc-test/src/transaction_test.cpp new file mode 100644 index 0000000000000..fb2b6c9ad9c9e --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/transaction_test.cpp @@ -0,0 +1,738 @@ +/* + * 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. + */ + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignition.h" + +#include "test_utils.h" +#include "odbc_test_suite.h" + +using namespace ignite; +using namespace ignite_test; + +using namespace boost::unit_test; + +/** + * Test setup fixture. + */ +struct TransactionTestSuiteFixture : public odbc::OdbcTestSuite +{ + static Ignite StartAdditionalNode(const char* name) + { + return StartTestNode("queries-transaction.xml", name); + } + + /** + * Constructor. + */ + TransactionTestSuiteFixture() : + grid(StartAdditionalNode("NodeMain")) + { + // No-op. + } + + /** + * Destructor. + */ + ~TransactionTestSuiteFixture() + { + // No-op. + } + + /** + * Insert test string value in cache and make all the neccessary checks. + * + * @param key Key. + * @param value Value. + */ + void InsertTestValue(int64_t key, const std::string& value) + { + SQLCHAR insertReq[] = "INSERT INTO TestType(_key, strField) VALUES(?, ?)"; + + SQLRETURN ret; + + ret = SQLPrepare(stmt, insertReq, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR, sizeof(strField), + sizeof(strField), &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + strncpy(strField, value.c_str(), sizeof(strField)); + strFieldLen = SQL_NTS; + + 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, 1); + + ret = SQLMoreResults(stmt); + + if (ret != SQL_NO_DATA) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ResetStatement(); + } + + /** + * Update test string value in cache and make all the neccessary checks. + * + * @param key Key. + * @param value Value. + */ + void UpdateTestValue(int64_t key, const std::string& value) + { + SQLCHAR insertReq[] = "UPDATE TestType SET strField=? WHERE _key=?"; + + SQLRETURN ret; + + ret = SQLPrepare(stmt, insertReq, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR, sizeof(strField), + sizeof(strField), &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + strncpy(strField, value.c_str(), sizeof(strField)); + strFieldLen = SQL_NTS; + + 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, 1); + + ret = SQLMoreResults(stmt); + + if (ret != SQL_NO_DATA) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ResetStatement(); + } + + /** + * Delete test string value. + * + * @param key Key. + */ + void DeleteTestValue(int64_t key) + { + SQLCHAR insertReq[] = "DELETE FROM TestType WHERE _key=?"; + + SQLRETURN ret; + + ret = SQLPrepare(stmt, insertReq, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0); + + 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)); + + SQLLEN affected = 0; + ret = SQLRowCount(stmt, &affected); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(affected, 1); + + ret = SQLMoreResults(stmt); + + if (ret != SQL_NO_DATA) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ResetStatement(); + } + + /** + * Selects and checks the value. + * + * @param key Key. + * @param expect Expected value. + */ + void CheckTestValue(int64_t key, const std::string& expect) + { + // Just selecting everything to make sure everything is OK + SQLCHAR selectReq[] = "SELECT strField FROM TestType WHERE _key = ?"; + + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq)); + + 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(std::string(strField, strFieldLen), expect); + + ret = SQLFetch(stmt); + + BOOST_CHECK_EQUAL(ret, SQL_NO_DATA); + + ret = SQLMoreResults(stmt); + + if (ret != SQL_NO_DATA) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ResetStatement(); + } + + /** + * Selects and checks that value is absent. + * + * @param key Key. + */ + void CheckNoTestValue(int64_t key) + { + // Just selecting everything to make sure everything is OK + SQLCHAR selectReq[] = "SELECT strField FROM TestType WHERE _key = ?"; + + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq)); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + BOOST_CHECK_EQUAL(ret, SQL_NO_DATA); + + ret = SQLMoreResults(stmt); + + if (ret != SQL_NO_DATA) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ResetStatement(); + } + + /** + * Reset statement state. + */ + void ResetStatement() + { + SQLRETURN ret = SQLFreeStmt(stmt, SQL_RESET_PARAMS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFreeStmt(stmt, SQL_UNBIND); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + } + + /** Node started during the test. */ + Ignite grid; +}; + +BOOST_FIXTURE_TEST_SUITE(TransactionTestSuite, TransactionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TransactionConnectionCommit) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_COMMIT); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionRollbackInsert) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckNoTestValue(42); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionRollbackUpdate1) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + UpdateTestValue(42, "Other"); + + CheckTestValue(42, "Other"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionRollbackUpdate2) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_COMMIT); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); + + UpdateTestValue(42, "Other"); + + CheckTestValue(42, "Other"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionRollbackDelete1) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + DeleteTestValue(42); + + CheckNoTestValue(42); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionRollbackDelete2) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_COMMIT); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); + + DeleteTestValue(42); + + CheckNoTestValue(42); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionTxModeError) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache;nested_tx_mode=error"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + ret = ExecQuery("BEGIN"); + + BOOST_CHECK_EQUAL(ret, SQL_ERROR); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionTxModeIgnore) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache;nested_tx_mode=ignore"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + ret = ExecQuery("BEGIN"); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckNoTestValue(42); +} + +BOOST_AUTO_TEST_CASE(TransactionConnectionTxModeCommit) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache;nested_tx_mode=commit"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + ret = ExecQuery("BEGIN"); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + UpdateTestValue(42, "Other"); + + CheckTestValue(42, "Other"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentCommit) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_COMMIT); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentRollbackInsert) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_ENV, env, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckNoTestValue(42); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentRollbackUpdate1) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + UpdateTestValue(42, "Other"); + + CheckTestValue(42, "Other"); + + ret = SQLEndTran(SQL_HANDLE_ENV, env, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentRollbackUpdate2) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_COMMIT); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); + + UpdateTestValue(42, "Other"); + + CheckTestValue(42, "Other"); + + ret = SQLEndTran(SQL_HANDLE_ENV, env, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentRollbackDelete1) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + DeleteTestValue(42); + + CheckNoTestValue(42); + + ret = SQLEndTran(SQL_HANDLE_ENV, env, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentRollbackDelete2) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + CheckTestValue(42, "Some"); + + ret = SQLEndTran(SQL_HANDLE_DBC, dbc, SQL_COMMIT); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); + + DeleteTestValue(42); + + CheckNoTestValue(42); + + ret = SQLEndTran(SQL_HANDLE_ENV, env, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentTxModeError) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache;nested_tx_mode=error"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + ret = ExecQuery("BEGIN"); + + BOOST_CHECK_EQUAL(ret, SQL_ERROR); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentTxModeIgnore) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache;nested_tx_mode=ignore"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + ret = ExecQuery("BEGIN"); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + ret = SQLEndTran(SQL_HANDLE_ENV, env, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckNoTestValue(42); +} + +BOOST_AUTO_TEST_CASE(TransactionEnvironmentTxModeCommit) +{ + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache;nested_tx_mode=commit"); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_AUTOCOMMIT, SQL_AUTOCOMMIT_OFF, 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + InsertTestValue(42, "Some"); + + ret = ExecQuery("BEGIN"); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + UpdateTestValue(42, "Other"); + + CheckTestValue(42, "Other"); + + ret = SQLEndTran(SQL_HANDLE_ENV, env, SQL_ROLLBACK); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + CheckTestValue(42, "Some"); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index 90e4dc99b7583..f706f4fab474c 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -83,6 +83,7 @@ libignite_odbc_la_SOURCES = \ src/protocol_version.cpp \ src/result_page.cpp \ src/row.cpp \ + src/nested_tx_mode.cpp \ src/message.cpp \ src/column.cpp \ src/statement.cpp \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h index c19f29d78dc7a..27de7e8cf3a95 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h @@ -26,6 +26,7 @@ #include "ignite/odbc/config/settable_value.h" #include "ignite/odbc/ssl/ssl_mode.h" #include "ignite/odbc/end_point.h" +#include "ignite/odbc/nested_tx_mode.h" namespace ignite { @@ -104,6 +105,9 @@ namespace ignite /** Default value for password attribute. */ static const std::string password; + + /** Default value for nestedTxMode attribute. */ + static const NestedTxMode::Type nestedTxMode; }; /** @@ -536,6 +540,27 @@ namespace ignite */ bool IsPasswordSet() const; + /** + * Get nested transaction mode. + * + * @return Nested transaction mode. + */ + NestedTxMode::Type GetNestedTxMode() const; + + /** + * Set nested transaction mode. + * + * @param mode Nested transaction mode. + */ + void SetNestedTxMode(NestedTxMode::Type mode); + + /** + * Check if the value set. + * + * @return @true if the value set. + */ + bool IsNestedTxModeSet() const; + /** * Get argument map. * @@ -613,6 +638,9 @@ namespace ignite /** Password. */ SettableValue password; + + /** Nested transaction mode. */ + SettableValue nestedTxMode; }; template<> @@ -642,6 +670,10 @@ namespace ignite template<> void Configuration::AddToMap(ArgumentMap& map, const std::string& key, const SettableValue& value); + + template<> + void Configuration::AddToMap(ArgumentMap& map, const std::string& key, + const SettableValue& value); } } } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_string_parser.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_string_parser.h index 2bfe331302f5a..605109edf8d4a 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_string_parser.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_string_parser.h @@ -97,6 +97,9 @@ namespace ignite /** Connection attribute keyword for password attribute. */ static const std::string password; + + /** Connection attribute keyword for nestedTxMode attribute. */ + static const std::string nestedTxMode; }; /** diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h index 68d781634b5ef..b3ae5fcbb9d62 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h @@ -33,6 +33,7 @@ namespace ignite { namespace odbc { + class Environment; class Statement; /** @@ -98,6 +99,11 @@ namespace ignite */ void Release(); + /** + * Deregister self from the parent. + */ + void Deregister(); + /** * Create statement associated with the connection. * @@ -140,6 +146,13 @@ namespace ignite */ const config::Configuration& GetConfiguration() const; + /** + * Is auto commit. + * + * @return @c true if the auto commit is enabled. + */ + bool IsAutoCommit(); + /** * Create diagnostic record associated with the Connection instance. * @@ -432,7 +445,10 @@ namespace ignite /** * Constructor. */ - Connection(); + Connection(Environment* env); + + /** Parent. */ + Environment* env; /** Client Socket. */ std::auto_ptr socket; @@ -443,6 +459,9 @@ namespace ignite /** Login timeout in seconds. */ int32_t loginTimeout; + /** Autocommit flag. */ + bool autoCommit; + /** Message parser. */ Parser parser; 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 46d79cec761ee..613cbf488808d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h @@ -82,6 +82,13 @@ namespace ignite */ virtual void AddStatusRecord(const OdbcError& err) = 0; + /** + * Add new status record. + * + * @param rec Record. + */ + virtual void AddStatusRecord(const DiagnosticRecord& rec) = 0; + protected: /** * Default constructor. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h index f1e9164c7bbec..d8cfeb71dd848 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h @@ -69,7 +69,7 @@ namespace ignite * * @return Diagnostic record. */ - virtual const diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() const + virtual const DiagnosticRecordStorage& GetDiagnosticRecords() const { return diagnosticRecords; } @@ -79,7 +79,7 @@ namespace ignite * * @return Diagnostic record. */ - virtual diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() + virtual DiagnosticRecordStorage& GetDiagnosticRecords() { return diagnosticRecords; } @@ -110,6 +110,13 @@ namespace ignite */ virtual void AddStatusRecord(const OdbcError& err); + /** + * Add new status record. + * + * @param rec Record. + */ + virtual void AddStatusRecord(const DiagnosticRecord& rec); + protected: /** Diagnostic records. */ DiagnosticRecordStorage diagnosticRecords; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/environment.h b/modules/platforms/cpp/odbc/include/ignite/odbc/environment.h index 3eeade6854053..e6a171fcebd4c 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/environment.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/environment.h @@ -18,6 +18,8 @@ #ifndef _IGNITE_ODBC_ENVIRONMENT #define _IGNITE_ODBC_ENVIRONMENT +#include + #include "ignite/odbc/diagnostic/diagnosable_adapter.h" namespace ignite @@ -32,6 +34,9 @@ namespace ignite class Environment : public diagnostic::DiagnosableAdapter { public: + /** Connection set type. */ + typedef std::set ConnectionSet; + /** * Constructor. */ @@ -49,6 +54,13 @@ namespace ignite */ Connection* CreateConnection(); + /** + * Deregister connection. + * + * @param conn Connection to deregister. + */ + void DeregisterConnection(Connection* conn); + /** * Perform transaction commit on all the associated connections. */ @@ -125,6 +137,9 @@ namespace ignite */ SqlResult::Type InternalGetAttribute(int32_t attr, app::ApplicationDataBuffer& buffer); + /** Assotiated connections. */ + ConnectionSet connections; + /** ODBC version. */ int32_t odbcVersion; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h index e119d1f00c477..87ba064883b2b 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h @@ -109,9 +109,10 @@ namespace ignite * @param sql SQL query. * @param params Query arguments. * @param timeout Timeout. + * @param autoCommit Auto commit flag. */ QueryExecuteRequest(const std::string& schema, const std::string& sql, const app::ParameterSet& params, - int32_t timeout); + int32_t timeout, bool autoCommit); /** * Destructor. @@ -137,6 +138,9 @@ namespace ignite /** Timeout. */ int32_t timeout; + + /** Auto commit. */ + bool autoCommit; }; /** @@ -154,9 +158,11 @@ namespace ignite * @param begin Beginng of the interval. * @param end End of the interval. * @param timeout Timeout. + * @param autoCommit Auto commit flag. */ QueryExecuteBatchtRequest(const std::string& schema, const std::string& sql, - const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last, int32_t timeout); + const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last, int32_t timeout, + bool autoCommit); /** * Destructor. @@ -191,6 +197,9 @@ namespace ignite /** Timeout. */ int32_t timeout; + + /** Auto commit. */ + bool autoCommit; }; /** diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/nested_tx_mode.h b/modules/platforms/cpp/odbc/include/ignite/odbc/nested_tx_mode.h new file mode 100644 index 0000000000000..eb2598d04fcae --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/nested_tx_mode.h @@ -0,0 +1,81 @@ +/* + * 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_NESTED_TX_MODE +#define _IGNITE_ODBC_NESTED_TX_MODE + +#include +#include + +namespace ignite +{ + namespace odbc + { + /** + * Nested transaction mode. + */ + struct NestedTxMode + { + /** + * Values. + */ + enum Type + { + /** Commit current transaction if a new one started. */ + AI_COMMIT = 1, + + /** Ignore start of a new transaction. */ + AI_IGNORE = 2, + + /** Throw an error. */ + AI_ERROR = 3, + + /** Returned when value is unknown. */ + AI_UNKNOWN = 100 + }; + + /** Mode set type. */ + typedef std::set ModeSet; + + /** + * Get value from a string value. + * + * @param str String. + * @param dflt Value to return on error. + * @return Corresponding value on success and @c dflt on failure. + */ + static Type FromString(const std::string& str, Type dflt = AI_UNKNOWN); + + /** + * Convert value to a string. + * + * @param value Value. + * @return String value. + */ + static std::string ToString(Type value); + + /** + * Get set of all valid values. + * + * @return Set of all valid values. + */ + static const ModeSet& GetValidValues(); + }; + } +} + +#endif //_IGNITE_ODBC_NESTED_TX_MODE \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h index 0b645364464df..1a722e31b983a 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h @@ -43,7 +43,7 @@ namespace ignite /** Version 2.3.2: added multiple statements support. */ static const ProtocolVersion VERSION_2_3_2; - /** Version 2.5.0: added multiple statements support. */ + /** Version 2.5.0: added authentication and transactions support. */ static const ProtocolVersion VERSION_2_5_0; /** Version 2.7.0: added fields precision and scale. */ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h index 1666863a7d50a..77974ade38138 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h @@ -61,6 +61,8 @@ namespace ignite SKIP_REDUCER_ON_UPDATE_CHECK_BOX, PROTOCOL_VERSION_LABEL, PROTOCOL_VERSION_COMBO_BOX, + NESTED_TX_MODE_LABEL, + NESTED_TX_MODE_COMBO_BOX, SSL_MODE_LABEL, SSL_MODE_COMBO_BOX, SSL_KEY_FILE_LABEL, @@ -312,6 +314,12 @@ namespace ignite /** Password edit. */ std::auto_ptr passwordEdit; + /** Nested transaction mode label. */ + std::auto_ptr nestedTxModeLabel; + + /** Nested transaction mode combo box. */ + std::auto_ptr nestedTxModeComboBox; + /** Configuration. */ config::Configuration& config; diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp index a94b479e98c58..e3bbb4ea97166 100644 --- a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp @@ -36,7 +36,7 @@ namespace ignite DsnConfigurationWindow::DsnConfigurationWindow(Window* parent, config::Configuration& config): CustomWindow(parent, "IgniteConfigureDsn", "Configure Apache Ignite DSN"), width(360), - height(580), + height(600), connectionSettingsGroupBox(), sslSettingsGroupBox(), authSettingsGroupBox(), @@ -59,6 +59,7 @@ namespace ignite userEdit(), passwordLabel(), passwordEdit(), + nestedTxModeComboBox(), okButton(), cancelButton(), config(config), @@ -286,7 +287,7 @@ namespace ignite int DsnConfigurationWindow::CreateAdditionalSettingsGroup(int posX, int posY, int sizeX) { - enum { LABEL_WIDTH = 80 }; + enum { LABEL_WIDTH = 130 }; int labelPosX = posX + INTERVAL; @@ -309,6 +310,29 @@ namespace ignite rowPos += INTERVAL + ROW_HEIGHT; + nestedTxModeLabel = CreateLabel(labelPosX, rowPos, LABEL_WIDTH, ROW_HEIGHT, + "Nested Transaction Mode:", ChildId::NESTED_TX_MODE_LABEL); + nestedTxModeComboBox = CreateComboBox(editPosX, rowPos, editSizeX, ROW_HEIGHT, + "", ChildId::NESTED_TX_MODE_COMBO_BOX); + + int id = 0; + + const NestedTxMode::ModeSet& supported = NestedTxMode::GetValidValues(); + + for (NestedTxMode::ModeSet::const_iterator it = supported.begin(); it != supported.end(); ++it) + { + nestedTxModeComboBox->AddString(NestedTxMode::ToString(*it)); + + if (*it == config.GetNestedTxMode()) + nestedTxModeComboBox->SetSelection(id); + + ++id; + } + + nestedTxModeComboBox->SetEnabled(version >= ProtocolVersion::VERSION_2_5_0); + + rowPos += INTERVAL + ROW_HEIGHT; + distributedJoinsCheckBox = CreateCheckBox(labelPosX, rowPos, checkBoxSize, ROW_HEIGHT, "Distributed Joins", ChildId::DISTRIBUTED_JOINS_CHECK_BOX, config.IsDistributedJoins()); @@ -429,6 +453,7 @@ namespace ignite ProtocolVersion version = ProtocolVersion::FromString(versionStr); lazyCheckBox->SetEnabled(version >= ProtocolVersion::VERSION_2_1_5); skipReducerOnUpdateCheckBox->SetEnabled(version >= ProtocolVersion::VERSION_2_3_0); + nestedTxModeComboBox->SetEnabled(version >= ProtocolVersion::VERSION_2_5_0); break; } @@ -568,13 +593,6 @@ namespace ignite { std::string pageSizeStr; - bool distributedJoins; - bool enforceJoinOrder; - bool replicatedOnly; - bool collocated; - bool lazy; - bool skipReducerOnUpdate; - pageSizeEdit->GetText(pageSizeStr); int32_t pageSize = common::LexicalCast(pageSizeStr); @@ -582,15 +600,22 @@ namespace ignite if (pageSize <= 0) pageSize = config.GetPageSize(); - distributedJoins = distributedJoinsCheckBox->IsChecked(); - enforceJoinOrder = enforceJoinOrderCheckBox->IsChecked(); - replicatedOnly = replicatedOnlyCheckBox->IsChecked(); - collocated = collocatedCheckBox->IsChecked(); - lazy = lazyCheckBox->IsChecked(); - skipReducerOnUpdate = skipReducerOnUpdateCheckBox->IsChecked(); + std::string nestedTxModeStr; + + nestedTxModeComboBox->GetText(nestedTxModeStr); + + NestedTxMode::Type mode = NestedTxMode::FromString(nestedTxModeStr, config.GetNestedTxMode()); + + bool distributedJoins = distributedJoinsCheckBox->IsChecked(); + bool enforceJoinOrder = enforceJoinOrderCheckBox->IsChecked(); + bool replicatedOnly = replicatedOnlyCheckBox->IsChecked(); + bool collocated = collocatedCheckBox->IsChecked(); + bool lazy = lazyCheckBox->IsChecked(); + bool skipReducerOnUpdate = skipReducerOnUpdateCheckBox->IsChecked(); LOG_MSG("Retrieving arguments:"); LOG_MSG("Page size: " << pageSize); + LOG_MSG("Nested TX Mode: " << NestedTxMode::ToString(mode)); LOG_MSG("Distributed Joins: " << (distributedJoins ? "true" : "false")); LOG_MSG("Enforce Join Order: " << (enforceJoinOrder ? "true" : "false")); LOG_MSG("Replicated only: " << (replicatedOnly ? "true" : "false")); @@ -599,6 +624,7 @@ namespace ignite LOG_MSG("Skip reducer on update: " << (skipReducerOnUpdate ? "true" : "false")); cfg.SetPageSize(pageSize); + cfg.SetNestedTxMode(mode); cfg.SetDistributedJoins(distributedJoins); cfg.SetEnforceJoinOrder(enforceJoinOrder); cfg.SetReplicatedOnly(replicatedOnly); diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index 352913efae505..630dc27a28d30 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -182,6 +182,7 @@ + @@ -230,6 +231,7 @@ + diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters index b1956aab3baa7..6da01112dd812 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters @@ -166,6 +166,9 @@ Code\config + + Code + @@ -329,5 +332,8 @@ Code + + Code + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index ef102d23208e3..a99894dfc7692 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -56,6 +56,8 @@ namespace ignite const std::string Configuration::DefaultValue::user = ""; const std::string Configuration::DefaultValue::password = ""; + const NestedTxMode::Type Configuration::DefaultValue::nestedTxMode = NestedTxMode::AI_ERROR; + Configuration::Configuration() : dsn(DefaultValue::dsn), driver(DefaultValue::driver), @@ -76,7 +78,8 @@ namespace ignite sslCertFile(DefaultValue::sslCertFile), sslCaFile(DefaultValue::sslCaFile), user(DefaultValue::user), - password(DefaultValue::password) + password(DefaultValue::password), + nestedTxMode(DefaultValue::nestedTxMode) { // No-op. } @@ -404,6 +407,21 @@ namespace ignite return password.IsSet(); } + NestedTxMode::Type Configuration::GetNestedTxMode() const + { + return nestedTxMode.GetValue(); + } + + void Configuration::SetNestedTxMode(NestedTxMode::Type mode) + { + this->nestedTxMode.SetValue(mode); + } + + bool Configuration::IsNestedTxModeSet() const + { + return nestedTxMode.IsSet(); + } + int32_t Configuration::GetPageSize() const { return pageSize.GetValue(); @@ -431,6 +449,7 @@ namespace ignite AddToMap(res, ConnectionStringParser::Key::sslCaFile, sslCaFile); AddToMap(res, ConnectionStringParser::Key::user, user); AddToMap(res, ConnectionStringParser::Key::password, password); + AddToMap(res, ConnectionStringParser::Key::nestedTxMode, nestedTxMode); } template<> @@ -486,6 +505,14 @@ namespace ignite if (value.IsSet()) map[key] = ssl::SslMode::ToString(value.GetValue()); } + + template<> + void Configuration::AddToMap(ArgumentMap& map, const std::string& key, + const SettableValue& value) + { + if (value.IsSet()) + map[key] = NestedTxMode::ToString(value.GetValue()); + } } } } diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index 7279ea67ab1ab..5885381b0532b 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -766,7 +766,7 @@ namespace ignite #ifdef SQL_MULTIPLE_ACTIVE_TXN // A character string: "Y" if the driver supports more than one active transaction at the same time, // "N" if only one transaction can be active at any time. - strParams[SQL_MULTIPLE_ACTIVE_TXN] = "N"; + strParams[SQL_MULTIPLE_ACTIVE_TXN] = "Y"; #endif // SQL_MULTIPLE_ACTIVE_TXN #ifdef SQL_ORDER_BY_COLUMNS_IN_SELECT @@ -1596,7 +1596,7 @@ namespace ignite // SQL_TXN_REPEATABLE_READ = Dirty reads and nonrepeatable reads are not possible. Phantoms are possible // SQL_TXN_SERIALIZABLE = Transactions are serializable. Serializable transactions do not allow dirty // reads, nonrepeatable reads, or phantoms. - intParams[SQL_DEFAULT_TXN_ISOLATION] = 0; + intParams[SQL_DEFAULT_TXN_ISOLATION] = SQL_TXN_REPEATABLE_READ; #endif // SQL_DEFAULT_TXN_ISOLATION #ifdef SQL_DROP_ASSERTION @@ -2170,7 +2170,7 @@ namespace ignite // SQL_ATTR_TXN_ISOLATION attribute. For more information, see SQLSetConnectAttr Function. // An SQL-92 Entry level-conformant driver will always return SQL_TXN_SERIALIZABLE as supported. // A FIPS Transitional level-conformant driver will always return all of these options as supported. - intParams[SQL_TXN_ISOLATION_OPTION] = 0; + intParams[SQL_TXN_ISOLATION_OPTION] = SQL_TXN_REPEATABLE_READ; #endif // SQL_TXN_ISOLATION_OPTION #ifdef SQL_UNION @@ -2312,7 +2312,7 @@ namespace ignite #ifdef SQL_TXN_CAPABLE // Describs the transaction support in the driver or data source. - shortParams[SQL_TXN_CAPABLE] = SQL_TC_NONE; + shortParams[SQL_TXN_CAPABLE] = SQL_TC_DDL_COMMIT; #endif // SQL_TXN_CAPABLE #ifdef SQL_QUOTED_IDENTIFIER_CASE diff --git a/modules/platforms/cpp/odbc/src/config/connection_string_parser.cpp b/modules/platforms/cpp/odbc/src/config/connection_string_parser.cpp index d23c2cb6052e7..7f7c2f4795404 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_string_parser.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_string_parser.cpp @@ -24,6 +24,7 @@ #include "ignite/odbc/ssl/ssl_mode.h" #include "ignite/odbc/config/connection_string_parser.h" #include "ignite/odbc/config/config_tools.h" +#include "ignite/odbc/nested_tx_mode.h" namespace ignite { @@ -51,6 +52,7 @@ namespace ignite const std::string ConnectionStringParser::Key::sslCaFile = "ssl_ca_file"; const std::string ConnectionStringParser::Key::user = "user"; const std::string ConnectionStringParser::Key::password = "password"; + const std::string ConnectionStringParser::Key::nestedTxMode = "nested_tx_mode"; ConnectionStringParser::ConnectionStringParser(Configuration& cfg): cfg(cfg) @@ -424,6 +426,23 @@ namespace ignite { cfg.SetPassword(value); } + else if (lKey == Key::nestedTxMode) + { + NestedTxMode::Type mode = NestedTxMode::FromString(value); + + if (mode == NestedTxMode::AI_UNKNOWN) + { + if (diag) + { + diag->AddStatusRecord(SqlState::S01S02_OPTION_VALUE_CHANGED, + "Specified nested transaction mode is not supported. Default value used ('error')."); + } + + return; + } + + cfg.SetNestedTxMode(mode); + } else if (diag) { std::stringstream stream; diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index 894efb7b20999..553854955f632 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -25,6 +25,7 @@ #include "ignite/odbc/log.h" #include "ignite/odbc/utility.h" +#include "ignite/odbc/environment.h" #include "ignite/odbc/statement.h" #include "ignite/odbc/connection.h" #include "ignite/odbc/message.h" @@ -54,10 +55,12 @@ namespace ignite { namespace odbc { - Connection::Connection() : + Connection::Connection(Environment* env) : + env(env), socket(), timeout(0), loginTimeout(SocketClient::DEFALT_CONNECT_TIMEOUT), + autoCommit(true), parser(), config(), info(config) @@ -186,6 +189,11 @@ namespace ignite IGNITE_ODBC_API_CALL(InternalRelease()); } + void Connection::Deregister() + { + env->DeregisterConnection(this); + } + SqlResult::Type Connection::InternalRelease() { if (socket.get() == 0) @@ -367,6 +375,11 @@ namespace ignite return config; } + bool Connection::IsAutoCommit() + { + return autoCommit; + } + diagnostic::DiagnosticRecord Connection::CreateStatusRecord(SqlState::Type sqlState, const std::string& message, int32_t rowNum, int32_t columnNum) { @@ -380,6 +393,37 @@ namespace ignite SqlResult::Type Connection::InternalTransactionCommit() { + std::string schema = config.GetSchema(); + + app::ParameterSet empty; + + QueryExecuteRequest req(schema, "COMMIT", empty, timeout, autoCommit); + QueryExecuteResponse rsp; + + try + { + bool sent = SyncMessage(req, rsp, timeout); + + if (!sent) + { + AddStatusRecord(SqlState::S08S01_LINK_FAILURE, "Failed to send commit request."); + + return SqlResult::AI_ERROR; + } + } + catch (const OdbcError& err) + { + AddStatusRecord(err); + + return SqlResult::AI_ERROR; + } + catch (const IgniteError& err) + { + AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText()); + + return SqlResult::AI_ERROR; + } + return SqlResult::AI_SUCCESS; } @@ -390,10 +434,38 @@ namespace ignite SqlResult::Type Connection::InternalTransactionRollback() { - AddStatusRecord(SqlState::SHYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Rollback operation is not supported."); + std::string schema = config.GetSchema(); + + app::ParameterSet empty; - return SqlResult::AI_ERROR; + QueryExecuteRequest req(schema, "ROLLBACK", empty, timeout, autoCommit); + QueryExecuteResponse rsp; + + try + { + bool sent = SyncMessage(req, rsp, timeout); + + if (!sent) + { + AddStatusRecord(SqlState::S08S01_LINK_FAILURE, "Failed to send rollback request."); + + return SqlResult::AI_ERROR; + } + } + catch (const OdbcError& err) + { + AddStatusRecord(err); + + return SqlResult::AI_ERROR; + } + catch (const IgniteError& err) + { + AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, err.GetText()); + + return SqlResult::AI_ERROR; + } + + return SqlResult::AI_SUCCESS; } void Connection::GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen) @@ -452,7 +524,10 @@ namespace ignite { SQLUINTEGER *val = reinterpret_cast(buf); - *val = SQL_AUTOCOMMIT_ON; + *val = autoCommit ? SQL_AUTOCOMMIT_ON : SQL_AUTOCOMMIT_OFF; + + if (valueLen) + *valueLen = SQL_IS_INTEGER; break; } @@ -507,10 +582,17 @@ namespace ignite case SQL_ATTR_AUTOCOMMIT: { - SQLUINTEGER val = static_cast(reinterpret_cast(value)); + SQLUINTEGER mode = static_cast(reinterpret_cast(value)); + + if (mode != SQL_AUTOCOMMIT_ON && mode != SQL_AUTOCOMMIT_OFF) + { + AddStatusRecord(SqlState::SHYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Specified attribute is not supported."); - if (val != SQL_AUTOCOMMIT_ON) return SqlResult::AI_ERROR; + } + + autoCommit = mode == SQL_AUTOCOMMIT_ON; break; } diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp index 143d949124a2a..6de071615c9c6 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp @@ -52,6 +52,11 @@ namespace ignite { AddStatusRecord(err.GetStatus(), err.GetErrorMessage(), 0, 0); } + + void DiagnosableAdapter::AddStatusRecord(const DiagnosticRecord& rec) + { + diagnosticRecords.AddStatusRecord(rec); + } } } } diff --git a/modules/platforms/cpp/odbc/src/dsn_config.cpp b/modules/platforms/cpp/odbc/src/dsn_config.cpp index 1307d7ef5fe4d..dcdb8f4392b92 100644 --- a/modules/platforms/cpp/odbc/src/dsn_config.cpp +++ b/modules/platforms/cpp/odbc/src/dsn_config.cpp @@ -212,6 +212,11 @@ namespace ignite if (password.IsSet() && !config.IsPasswordSet()) config.SetPassword(password.GetValue()); + + SettableValue nestedTxModeStr = ReadDsnString(dsn, ConnectionStringParser::Key::nestedTxMode); + + if (nestedTxModeStr.IsSet() && !config.IsNestedTxModeSet()) + config.SetNestedTxMode(NestedTxMode::FromString(nestedTxModeStr.GetValue(), config.GetNestedTxMode())); } } } \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/environment.cpp b/modules/platforms/cpp/odbc/src/environment.cpp index 417fedc5072c7..5a2ac4fc4a065 100644 --- a/modules/platforms/cpp/odbc/src/environment.cpp +++ b/modules/platforms/cpp/odbc/src/environment.cpp @@ -25,7 +25,8 @@ namespace ignite { namespace odbc { - Environment::Environment() : + Environment::Environment() : + connections(), odbcVersion(SQL_OV_ODBC3), odbcNts(SQL_TRUE) { @@ -46,9 +47,14 @@ namespace ignite return connection; } + void Environment::DeregisterConnection(Connection* conn) + { + connections.erase(conn); + } + SqlResult::Type Environment::InternalCreateConnection(Connection*& connection) { - connection = new Connection; + connection = new Connection(this); if (!connection) { @@ -57,6 +63,8 @@ namespace ignite return SqlResult::AI_ERROR; } + connections.insert(connection); + return SqlResult::AI_SUCCESS; } @@ -67,7 +75,25 @@ namespace ignite SqlResult::Type Environment::InternalTransactionCommit() { - return SqlResult::AI_SUCCESS; + SqlResult::Type res = SqlResult::AI_SUCCESS; + + for (ConnectionSet::iterator it = connections.begin(); it != connections.end(); ++it) + { + Connection* conn = *it; + + conn->TransactionCommit(); + + diagnostic::DiagnosticRecordStorage& diag = conn->GetDiagnosticRecords(); + + if (diag.GetStatusRecordsNumber() > 0) + { + AddStatusRecord(diag.GetStatusRecord(1)); + + res = SqlResult::AI_SUCCESS_WITH_INFO; + } + } + + return res; } void Environment::TransactionRollback() @@ -77,10 +103,25 @@ namespace ignite SqlResult::Type Environment::InternalTransactionRollback() { - AddStatusRecord(SqlState::SHYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Rollback operation is not supported."); + SqlResult::Type res = SqlResult::AI_SUCCESS; - return SqlResult::AI_ERROR; + for (ConnectionSet::iterator it = connections.begin(); it != connections.end(); ++it) + { + Connection* conn = *it; + + conn->TransactionRollback(); + + diagnostic::DiagnosticRecordStorage& diag = conn->GetDiagnosticRecords(); + + if (diag.GetStatusRecordsNumber() > 0) + { + AddStatusRecord(diag.GetStatusRecord(1)); + + res = SqlResult::AI_SUCCESS_WITH_INFO; + } + } + + return res; } void Environment::SetAttribute(int32_t attr, void* value, int32_t len) diff --git a/modules/platforms/cpp/odbc/src/message.cpp b/modules/platforms/cpp/odbc/src/message.cpp index 5b909b8fda2ae..57308424a04b2 100644 --- a/modules/platforms/cpp/odbc/src/message.cpp +++ b/modules/platforms/cpp/odbc/src/message.cpp @@ -82,15 +82,18 @@ namespace ignite { utility::WriteString(writer, config.GetUser()); utility::WriteString(writer, config.GetPassword()); + + writer.WriteInt8(config.GetNestedTxMode()); } } QueryExecuteRequest::QueryExecuteRequest(const std::string& schema, const std::string& sql, - const app::ParameterSet& params, int32_t timeout): + const app::ParameterSet& params, int32_t timeout, bool autoCommit): schema(schema), sql(sql), params(params), - timeout(timeout) + timeout(timeout), + autoCommit(autoCommit) { // No-op. } @@ -115,17 +118,21 @@ namespace ignite if (ver >= ProtocolVersion::VERSION_2_3_2) writer.WriteInt32(timeout); + + if (ver >= ProtocolVersion::VERSION_2_5_0) + writer.WriteBool(autoCommit); } QueryExecuteBatchtRequest::QueryExecuteBatchtRequest(const std::string& schema, const std::string& sql, - const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last, int32_t timeout): + const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last, int32_t timeout, bool autoCommit) : schema(schema), sql(sql), params(params), begin(begin), end(end), last(last), - timeout(timeout) + timeout(timeout), + autoCommit(autoCommit) { // No-op. } @@ -150,6 +157,9 @@ namespace ignite if (ver >= ProtocolVersion::VERSION_2_3_2) writer.WriteInt32(timeout); + + if (ver >= ProtocolVersion::VERSION_2_5_0) + writer.WriteBool(autoCommit); } QueryCloseRequest::QueryCloseRequest(int64_t queryId): queryId(queryId) diff --git a/modules/platforms/cpp/odbc/src/nested_tx_mode.cpp b/modules/platforms/cpp/odbc/src/nested_tx_mode.cpp new file mode 100644 index 0000000000000..e4e6118addc61 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/nested_tx_mode.cpp @@ -0,0 +1,80 @@ +/* + * 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/nested_tx_mode.h" +#include "ignite/common/utils.h" + +namespace +{ + using ignite::odbc::NestedTxMode; + NestedTxMode::Type validValues0[] = { + NestedTxMode::AI_COMMIT, + NestedTxMode::AI_IGNORE, + NestedTxMode::AI_ERROR + }; + + NestedTxMode::ModeSet validValues(validValues0, validValues0 + (sizeof(validValues0) / sizeof(validValues0[0]))); +} + + +namespace ignite +{ + namespace odbc + { + NestedTxMode::Type NestedTxMode::FromString(const std::string& str, Type dflt) + { + std::string lower = common::ToLower(str); + + if (lower == "commit") + return AI_COMMIT; + + if (lower == "ignore") + return AI_IGNORE; + + if (lower == "error") + return AI_ERROR; + + return dflt; + } + + std::string NestedTxMode::ToString(Type value) + { + switch (value) + { + case AI_COMMIT: + return "commit"; + + case AI_IGNORE: + return "ignore"; + + case AI_ERROR: + return "error"; + + default: + break; + } + + return "default"; + } + + const NestedTxMode::ModeSet& NestedTxMode::GetValidValues() + { + return validValues; + } + } +} + diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 3aad52f315432..5b37da3577da9 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -204,6 +204,8 @@ namespace ignite if (!connection) return SQL_INVALID_HANDLE; + connection->Deregister(); + delete connection; return SQL_SUCCESS; diff --git a/modules/platforms/cpp/odbc/src/query/batch_query.cpp b/modules/platforms/cpp/odbc/src/query/batch_query.cpp index 8dada3ca82774..1256c94c3ddce 100644 --- a/modules/platforms/cpp/odbc/src/query/batch_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/batch_query.cpp @@ -144,7 +144,9 @@ namespace ignite { const std::string& schema = connection.GetSchema(); - QueryExecuteBatchtRequest req(schema, sql, params, begin, end, last, timeout); + QueryExecuteBatchtRequest req(schema, sql, params, begin, end, last, timeout, + connection.IsAutoCommit()); + QueryExecuteBatchResponse rsp; try diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp index 6974c7f33c856..69208b84a5cbf 100644 --- a/modules/platforms/cpp/odbc/src/query/data_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp @@ -206,7 +206,7 @@ namespace ignite { const std::string& schema = connection.GetSchema(); - QueryExecuteRequest req(schema, sql, params, timeout); + QueryExecuteRequest req(schema, sql, params, timeout, connection.IsAutoCommit()); QueryExecuteResponse rsp; try diff --git a/modules/yardstick/config/mvcc/benchmark-mvcc-messages.sh b/modules/yardstick/config/mvcc/benchmark-mvcc-messages.sh new file mode 100644 index 0000000000000..32068d35332cf --- /dev/null +++ b/modules/yardstick/config/mvcc/benchmark-mvcc-messages.sh @@ -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. +# + +# +# Configuration to measure increased messages load with mvcc turned on. +# + +now0=`date +'%H%M%S'` + +# JVM options. +JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" + +# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. +JVM_OPTS=${JVM_OPTS}" \ +-Xms8g \ +-Xmx8g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ +-XX:+PrintGCDateStamps \ +" + +#Ignite version +ver="RELEASE-" + +# List of default probes. +# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). +BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe + +# Packages where the specified benchmark is searched by reflection mechanism. +BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick + +# Flag which indicates to restart the servers before every benchmark execution. +RESTART_SERVERS=true + +# Probe point writer class name. +# BENCHMARK_WRITER= + +# The benchmark is designed to run with 1 client node (driver itself) and many (4 for instance) server nodes. +SERVER_HOSTS=localhost,localhost,localhost,localhost +DRIVER_HOSTS=localhost + +# Remote username. +# REMOTE_USER= + +# Number of nodes, used to wait for the specified number of nodes to start. +nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`)) + +# Warmup. +w=30 + +# Duration. +d=300 + +# Threads count. +t=1 + +# Sync mode. +sm=FULL_SYNC + +# Parameters that should be the same across all the benchmarks launches. +commonParams="-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -w ${w} -d ${d} \ + -jdbc jdbc:ignite:thin://auto.find/ -t ${t} -sm ${sm} \ + --clientNodesAfterId 100 \ + -sn IgniteNode -cl --range 1000000 \ + --atomic-mode TRANSACTIONAL" + +# Run configuration which contains all benchmarks. +# Note that each benchmark is set to run for 300 seconds (5 min) with warm-up set to 60 seconds (1 minute). +CONFIGS="\ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1-backup-0-mvcc-off -b 0 --sqlRange 1, \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1-backup-0-mvcc-on -b 0 --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1000-backup-0-mvcc-off -b 0 --sqlRange 1000, \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1000-backup-0-mvcc-on -b 0 --sqlRange 1000 --mvcc, \ + \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1-backup-2-mvcc-off -b 2 --sqlRange 1, \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1-backup-2-mvcc-on -b 2 --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1000-backup-2-mvcc-off -b 2 --sqlRange 1000, \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1000-backup-2-mvcc-on -b 2 --sqlRange 1000 --mvcc, \ +" diff --git a/modules/yardstick/config/mvcc/benchmark-mvcc-processor.sh b/modules/yardstick/config/mvcc/benchmark-mvcc-processor.sh new file mode 100644 index 0000000000000..db4519927277a --- /dev/null +++ b/modules/yardstick/config/mvcc/benchmark-mvcc-processor.sh @@ -0,0 +1,95 @@ +# +# 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. +# + +# +# Configuration to compare mvcc on/off. This benchmark creates load on mvcc processor. +# + +now0=`date +'%H%M%S'` + +# JVM options. +JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" + +# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. +JVM_OPTS=${JVM_OPTS}" \ +-Xms8g \ +-Xmx8g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ +-XX:+PrintGCDateStamps \ +" + +#Ignite version +ver="RELEASE-" + +# List of default probes. +# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). +BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe + +# Packages where the specified benchmark is searched by reflection mechanism. +BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick + +# Flag which indicates to restart the servers before every benchmark execution. +RESTART_SERVERS=true + +# Probe point writer class name. +# BENCHMARK_WRITER= + +# The benchmark is designed to run with 4 client nodes (drivers) and 1 server node. +SERVER_HOSTS=localhost +DRIVER_HOSTS=localhost,localhost,localhost,localhost + +# Remote username. +# REMOTE_USER= + +# Number of nodes, used to wait for the specified number of nodes to start. +nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`)) + +# Warmup. +w=30 + +# Duration. +d=300 + +# Threads count. +t=16 + +# Sync mode. +sm=FULL_SYNC + +# Parameters that should be the same across all the benchmarks launches. +commonParams="-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -w ${w} -d ${d} \ + -jdbc jdbc:ignite:thin://auto.find/ -t ${t} -sm ${sm} \ + --clientNodesAfterId 100 \ + -sn IgniteNode -cl --range 1000000 \ + --atomic-mode TRANSACTIONAL" + +# Run configuration which contains all benchmarks. +# Note that each benchmark is set to run for 300 seconds (5 min) with warm-up set to 60 seconds (1 minute). +CONFIGS="\ +${commonParams} -dn MvccProcessorBenchmark -ds ${ver}sql-update-batch-1-backup-0-mvcc-off -b 0 --sqlRange 1, \ +${commonParams} -dn MvccProcessorBenchmark -ds ${ver}sql-update-batch-1-backup-0-mvcc-on -b 0 --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn MvccProcessorBenchmark -ds ${ver}sql-update-batch-25-backup-0-mvcc-off -b 0 --sqlRange 25, \ +${commonParams} -dn MvccProcessorBenchmark -ds ${ver}sql-update-batch-25-backup-0-mvcc-on -b 0 --sqlRange 25 --mvcc, \ + \ +${commonParams} -dn MvccProcessorBenchmark -ds ${ver}sql-update-batch-1000-backup-0-mvcc-off -b 0 --sqlRange 1000, \ +${commonParams} -dn MvccProcessorBenchmark -ds ${ver}sql-update-batch-1000-backup-0-mvcc-on -b 0 --sqlRange 1000 --mvcc \ +" diff --git a/modules/yardstick/config/mvcc/benchmark-mvcc-updates-contention.sh b/modules/yardstick/config/mvcc/benchmark-mvcc-updates-contention.sh new file mode 100644 index 0000000000000..0a92bcec40bd3 --- /dev/null +++ b/modules/yardstick/config/mvcc/benchmark-mvcc-updates-contention.sh @@ -0,0 +1,96 @@ +# +# 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. +# + +# +# Configuration to measure performance of concurrent sql updates with contention. +# Update keys are shared among the threads/hosts. +# +now0=`date +'%H%M%S'` + +# JVM options. +JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" + +# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. +JVM_OPTS=${JVM_OPTS}" \ +-Xms8g \ +-Xmx8g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ +-XX:+PrintGCDateStamps \ +" + +#Ignite version +ver="RELEASE-" + +# List of default probes. +# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). +BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe + +# Packages where the specified benchmark is searched by reflection mechanism. +BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick + +# Flag which indicates to restart the servers before every benchmark execution. +RESTART_SERVERS=true + +# Probe point writer class name. +# BENCHMARK_WRITER= + +# The benchmark is designed to run with 4 client node (drivers) and several (2 for instance) server nodes +SERVER_HOSTS=localhost,localhost +DRIVER_HOSTS=localhost,localhost,localhost,localhost + +# Remote username. +# REMOTE_USER= + +# Number of nodes, used to wait for the specified number of nodes to start. +nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`)) + +# Warmup. +w=30 + +# Duration. +d=300 + +# Threads count. +t=16 + +# Sync mode. +sm=FULL_SYNC + +# Parameters that should be the same across all the benchmarks launches. +commonParams="-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -w ${w} -d ${d} \ + -jdbc jdbc:ignite:thin://auto.find/ -t ${t} -sm ${sm} \ + --clientNodesAfterId 100 \ + -sn IgniteNode -cl \ + --range 1000000 --mvcc-contention-range 10000 \ + --atomic-mode TRANSACTIONAL" + +# Run configuration which contains all benchmarks. +# Note that each benchmark is set to run for 300 seconds (5 min) with warm-up set to 60 seconds (1 minute). +CONFIGS="\ +${commonParams} -dn MvccUpdateContentionBenchmark -ds ${ver}sql-update-batch-1-backup-0-mvcc-off -b 0 --sqlRange 1, \ +${commonParams} -dn MvccUpdateContentionBenchmark -ds ${ver}sql-update-batch-1-backup-0-mvcc-on -b 0 --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn MvccUpdateContentionBenchmark -ds ${ver}sql-update-batch-25-backup-0-mvcc-off -b 0 --sqlRange 25, \ +${commonParams} -dn MvccUpdateContentionBenchmark -ds ${ver}sql-update-batch-25-backup-0-mvcc-on -b 0 --sqlRange 25 --mvcc, \ + \ +${commonParams} -dn MvccUpdateContentionBenchmark -ds ${ver}sql-update-batch-1000-backup-0-mvcc-off -b 0 --sqlRange 1000, \ +${commonParams} -dn MvccUpdateContentionBenchmark -ds ${ver}sql-update-batch-1000-backup-0-mvcc-on -b 0 --sqlRange 1000 --mvcc \ +" diff --git a/modules/yardstick/config/mvcc/benchmark-thin-native.properties b/modules/yardstick/config/mvcc/benchmark-thin-native.properties new file mode 100644 index 0000000000000..3fae9a8884986 --- /dev/null +++ b/modules/yardstick/config/mvcc/benchmark-thin-native.properties @@ -0,0 +1,124 @@ +# +# 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. +# + +# +# Configuration to measure mvcc impact on jdbc operations. +# + +now0=`date +'%H%M%S'` + +# JVM options. +JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" + +# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. +JVM_OPTS=${JVM_OPTS}" \ +-Xms8g \ +-Xmx8g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ +-XX:+PrintGCDateStamps \ +" + +#Ignite version +ver="RELEASE-" + +# List of default probes. +# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). +BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe + +# Packages where the specified benchmark is searched by reflection mechanism. +BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick + +# Flag which indicates to restart the servers before every benchmark execution. +RESTART_SERVERS=true + +# Probe point writer class name. +# BENCHMARK_WRITER= + +# The benchmark is applicable only for 2 servers (the second server is started in client mode) and 1 driver. +SERVER_HOSTS=localhost,localhost +DRIVER_HOSTS=localhost + +# Remote username. +# REMOTE_USER= + +# Number of nodes, used to wait for the specified number of nodes to start. +nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`)) + +# Backups count. +b=1 + +# Warmup. +w=30 + +# Duration. +d=300 + +# Threads count. +t=4 + +# Sync mode. +sm=FULL_SYNC + +# Parameters that should be the same across all the benchmarks launches. +commonParams="-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} \ + -jdbc jdbc:ignite:thin://auto.find/ -t ${t} -sm ${sm} \ + --clientNodesAfterId 0 \ + -sn IgniteNode -cl --range 1000000 \ + --atomic-mode TRANSACTIONAL" + +# Run configuration which contains all benchmarks. +# Note that each benchmark is set to run for 300 seconds (5 min) with warm-up set to 60 seconds (1 minute). +CONFIGS="\ +${commonParams} -dn JdbcSqlInsertDeleteBenchmark -ds ${ver}sql-insert-delete-batch-1-jdbc-thin-mvcc-off, \ +${commonParams} -dn JdbcSqlInsertDeleteBenchmark -ds ${ver}sql-insert-delete-batch-1-jdbc-thin-mvcc-on --mvcc, \ + \ +${commonParams} -dn JdbcSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1-jdbc-thin-mvcc-off --sqlRange 1, \ +${commonParams} -dn JdbcSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1-jdbc-thin-mvcc-on --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn JdbcSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1000-jdbc-thin-mvcc-off --sqlRange 1000, \ +${commonParams} -dn JdbcSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1000-jdbc-thin-mvcc-on --sqlRange 1000 --mvcc, \ + \ +${commonParams} -dn JdbcSqlUpdateBenchmark -ds ${ver}sql-update-batch-1-jdbc-thin-mvcc-off --sqlRange 1, \ +${commonParams} -dn JdbcSqlUpdateBenchmark -ds ${ver}sql-update-batch-1-jdbc-thin-mvcc-on --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn JdbcSqlUpdateBenchmark -ds ${ver}sql-update-batch-1000-jdbc-thin-mvcc-off --sqlRange 1000, \ +${commonParams} -dn JdbcSqlUpdateBenchmark -ds ${ver}sql-update-batch-1000-jdbc-thin-mvcc-on --sqlRange 1000 --mvcc, \ + \ + \ +${commonParams} -dn NativeSqlInsertDeleteBenchmark -ds ${ver}sql-insert-delete-batch-1-native-sql-mvcc-off, \ +${commonParams} -dn NativeSqlInsertDeleteBenchmark -ds ${ver}sql-insert-delete-batch-1-native-sql-mvcc-on --mvcc, \ + \ +${commonParams} -dn NativeSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1-native-sql-mvcc-off --sqlRange 1, \ +${commonParams} -dn NativeSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1-native-sql-mvcc-on --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn NativeSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1000-native-sql-mvcc-off --sqlRange 1000, \ +${commonParams} -dn NativeSqlQueryRangeBenchmark -ds ${ver}sql-select-batch-1000-native-sql-mvcc-on --sqlRange 1000 --mvcc, \ + \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1-native-sql-mvcc-off --sqlRange 1, \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1-native-sql-mvcc-on --sqlRange 1 --mvcc, \ + \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1000-native-sql-mvcc-off --sqlRange 1000, \ +${commonParams} -dn NativeSqlUpdateRangeBenchmark -ds ${ver}sql-update-batch-1000-native-sql-mvcc-on --sqlRange 1000 --mvcc, \ + \ + \ +${commonParams} -dn NativeJavaApiPutRemoveBenchmark -ds ${ver}sql-update-batch-1-native-sql-mvcc-off, \ +${commonParams} -dn NativeJavaApiPutRemoveBenchmark -ds ${ver}sql-update-batch-1-native-sql-mvcc-on --mvcc \ +" diff --git a/modules/yardstick/config/upload/benchmark-jdbc-thin-inmemory-mvcc.properties b/modules/yardstick/config/upload/benchmark-jdbc-thin-inmemory-mvcc.properties new file mode 100644 index 0000000000000..6e55b9bdf2a7c --- /dev/null +++ b/modules/yardstick/config/upload/benchmark-jdbc-thin-inmemory-mvcc.properties @@ -0,0 +1,105 @@ +# +# 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. +# + +# +# Benchmarks for data upload in inmemory mode (persistence disabled) with and without mvcc. +# + +now0=`date +'%H%M%S'` + +# JVM options. +JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" + +# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. +JVM_OPTS=${JVM_OPTS}" \ +-Xms8g \ +-Xmx8g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ +-XX:+PrintGCDateStamps \ +" + +#Ignite version +ver="RELEASE-" + +# List of default probes. +# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). +BENCHMARK_DEFAULT_PROBES=TotalTimeProbe + +# Packages where the specified benchmark is searched by reflection mechanism. +BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick + +# Flag which indicates to restart the servers before every benchmark execution. +RESTART_SERVERS=true + +# Probe point writer class name. +# BENCHMARK_WRITER= + +# The benchmark is applicable only for 2 servers (the second server is started in client mode) and 1 driver. +SERVER_HOSTS=localhost,localhost +DRIVER_HOSTS=localhost + +# Remote username. +# REMOTE_USER= + +# Number of nodes, used to wait for the specified number of nodes to start. +nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`)) + +# Backups count. +b=1 + +# Warmup. +w=0 + +# Threads count. +t=1 + +# Sync mode. +sm=FULL_SYNC + +# Parameters that should be the same across all the benchmarks launches. +commonParams="\ +-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} \ + --warmup ${w} --operations 1 \ + -jdbc jdbc:ignite:thin://auto.find/ \ + --threads ${t} --syncMode ${sm} -sn IgniteNode \ + --upload-rows 1000000 -cl \ + --clientNodesAfterId 0 \ + --atomic-mode TRANSACTIONAL \ +" + +# Run configuration which contains all benchmarks. +# Note that each benchmark is set to run only one time, warmup parameter is set to 0 due to custom warmup operation. +CONFIGS="\ +${commonParams} -dn NativePutBenchmark -ds ${ver}upload-native-put-mvcc-off, \ +${commonParams} -dn NativePutBenchmark -ds ${ver}upload-native-put-mvcc-on --mvcc, \ + \ +${commonParams} -dn NativeStreamerBenchmark -ds ${ver}upload-native-streamer-mvcc-off --streamer-local-batch-size 1000, \ +${commonParams} -dn NativeStreamerBenchmark -ds ${ver}upload-native-streamer-mvcc-on --streamer-local-batch-size 1000 --mvcc, \ + \ +${commonParams} -dn CopyBenchmark -ds ${ver}upload-copy-mvcc-off, \ +${commonParams} -dn CopyBenchmark -ds ${ver}upload-copy-mvcc-on --mvcc, \ + \ +${commonParams} -dn InsertBenchmark -ds ${ver}upload-insert-mvcc-off, \ +${commonParams} -dn InsertBenchmark -ds ${ver}upload-insert-mvcc-on --mvcc, \ + \ +${commonParams} -dn BatchedInsertBenchmark -ds ${ver}upload-batched-insert-mvcc-off --upload-jdbc-batch-size 1000, \ +${commonParams} -dn BatchedInsertBenchmark -ds ${ver}upload-batched-insert-mvcc-on --upload-jdbc-batch-size 1000 --mvcc \ +" diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java index 72409a05e8b35..bbe2ab37e4ccd 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java @@ -21,7 +21,9 @@ import com.beust.jcommander.ParametersDelegate; import java.util.Collections; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.internal.util.tostring.GridToStringBuilder; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -59,6 +61,10 @@ public class IgniteBenchmarkArguments { @Parameter(names = {"-sm", "--syncMode"}, description = "Synchronization mode") private CacheWriteSynchronizationMode syncMode = CacheWriteSynchronizationMode.PRIMARY_SYNC; + /** */ + @Parameter(names = {"--atomic-mode", "--atomicMode"}) + @Nullable private CacheAtomicityMode atomicMode = null; + /** */ @Parameter(names = {"-cl", "--client"}, description = "Client flag") private boolean clientOnly = false; @@ -262,6 +268,17 @@ public class IgniteBenchmarkArguments { @Parameter(names = {"-stbs", "--streamerBufSize"}, description = "Data streamer buffer size") private int streamerBufSize = IgniteDataStreamer.DFLT_PER_NODE_BUFFER_SIZE; + /** */ + @Parameter(names = {"-mvcc", "--mvcc"}, description = "Enable MVCC for cache") + private boolean mvcc; + + /** + * @return {@code True} if need enable cache mvcc (see {@link CacheConfiguration#isMvccEnabled()}). + */ + public boolean mvccEnabled() { + return mvcc; + } + /** */ @Parameter(names = {"-sqlr", "--sqlRange"}, description = "Result set size") @GridToStringInclude @@ -277,6 +294,15 @@ public class IgniteBenchmarkArguments { @GridToStringInclude public UploadBenchmarkArguments upload = new UploadBenchmarkArguments(); + /** */ + @Parameter(names = {"--mvcc-contention-range", "--mvccContentionRange"}, + description = "Mvcc benchmark specific: " + + "Size of range of table keys that should be used in query. " + + "Should be less than 'range'. " + + "Useful together with 'sqlRange' to control, how often key contentions of sql operations occur.") + @GridToStringInclude + public long mvccContentionRange = 10_000; + /** * @return {@code True} if need set {@link DataStorageConfiguration}. */ @@ -389,6 +415,11 @@ public CacheWriteSynchronizationMode syncMode() { return syncMode; } + /** With what cache atomicity mode to create tables. */ + @Nullable public CacheAtomicityMode atomicMode(){ + return atomicMode; + } + /** * @return Backups. */ @@ -688,6 +719,13 @@ public int clientNodesAfterId() { return clientNodesAfterId; } + /** + * @return Mvcc contention range. + */ + public long mvccContentionRange() { + return mvccContentionRange; + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(IgniteBenchmarkArguments.class, this); 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 0c61435140034..a259cbf575748 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 @@ -103,6 +103,8 @@ public IgniteNode(boolean clientMode, Ignite ignite) { CacheConfiguration[] ccfgs = c.getCacheConfiguration(); + c.setMvccEnabled(args.mvccEnabled()); + if (ccfgs != null) { for (CacheConfiguration cc : ccfgs) { // IgniteNode can not run in CLIENT_ONLY mode, diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractJdbcBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractJdbcBenchmark.java index 6ab024fc5df49..7d5b9400a2032 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractJdbcBenchmark.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractJdbcBenchmark.java @@ -92,7 +92,7 @@ public abstract class AbstractJdbcBenchmark extends IgniteAbstractBenchmark { * @throws Exception On error. */ protected void setupData() throws Exception { - fillData(cfg, (IgniteEx)ignite(), args.range()); + fillData(cfg, (IgniteEx)ignite(), args.range(), args.atomicMode()); } /** diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java index 5e92514e371a5..129e6a460c011 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java @@ -32,6 +32,6 @@ public abstract class AbstractNativeBenchmark extends IgniteAbstractBenchmark { @Override public void setUp(BenchmarkConfiguration cfg) throws Exception { super.setUp(cfg); - fillData(cfg, (IgniteEx)ignite(), args.range()); + fillData(cfg, (IgniteEx)ignite(), args.range(), args.atomicMode()); } } diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java index 74b5da5e75f66..81d6c17258ed9 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java @@ -17,6 +17,7 @@ package org.apache.ignite.yardstick.jdbc; +import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.IgniteEx; import org.yardstickframework.BenchmarkConfiguration; @@ -33,11 +34,17 @@ public class JdbcUtils { * @param ignite Ignite node. * @param range Data key range. */ - static void fillData(BenchmarkConfiguration cfg, IgniteEx ignite, long range) { + public static void fillData(BenchmarkConfiguration cfg, IgniteEx ignite, long range, CacheAtomicityMode atomicMode) { println(cfg, "Create table..."); + String withExpr = atomicMode != null ? " WITH \"atomicity=" + atomicMode.name() + "\";" : ";"; + + String qry = "CREATE TABLE test_long (id long primary key, val long)" + withExpr; + + println(cfg, "Creating table with schema: " + qry); + ignite.context().query().querySqlFields( - new SqlFieldsQuery("CREATE TABLE test_long (id long primary key, val long)"), true); + new SqlFieldsQuery(qry), true); println(cfg, "Populate data..."); diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeJavaApiPutRemoveBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeJavaApiPutRemoveBenchmark.java new file mode 100644 index 0000000000000..c1106d74ede2a --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeJavaApiPutRemoveBenchmark.java @@ -0,0 +1,52 @@ +/* + * 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.jdbc; + +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.yardstickframework.BenchmarkConfiguration; + +/** + * Benchmark that fetches data from cache to compare with SQL SELECT operation. + */ +public class NativeJavaApiPutRemoveBenchmark extends AbstractNativeBenchmark { + /** Cache for created table. */ + private IgniteCache tabCache; + + @Override public void setUp(BenchmarkConfiguration cfg) throws Exception { + super.setUp(cfg); + + tabCache = ignite().cache("SQL_PUBLIC_TEST_LONG"); + } + + @Override public boolean test(Map ctx) throws Exception { + long insertKey = ThreadLocalRandom.current().nextLong(args.range()) + 1 + args.range(); + long insertVal = insertKey + 1; + + try { + tabCache.put(insertKey, insertVal); + tabCache.remove(insertKey); + } catch (IgniteException ign){ + // Collision occurred, ignoring. + } + + return true; + } +} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/AbstractDistributedMvccBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/AbstractDistributedMvccBenchmark.java new file mode 100644 index 0000000000000..daf50a1b079b1 --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/AbstractDistributedMvccBenchmark.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.yardstick.jdbc.mvcc; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.IgniteCountDownLatch; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.yardstick.IgniteAbstractBenchmark; +import org.yardstickframework.BenchmarkConfiguration; + +import static org.apache.ignite.yardstick.jdbc.JdbcUtils.fillData; +import static org.yardstickframework.BenchmarkUtils.println; + +/** + * Base for mvcc benchmarks that are running on multiply hosts. + */ +public abstract class AbstractDistributedMvccBenchmark extends IgniteAbstractBenchmark { + /** Sql query to create load. */ + public static final String UPDATE_QRY = "UPDATE test_long SET val = (val + 1) WHERE id BETWEEN ? AND ?"; + + /** Timeout in minutest for test data to be loaded. */ + public static final long DATA_WAIT_TIMEOUT_MIN = 20; + + /** Member id of the host driver is running */ + protected int memberId; + /** + * Number of nodes handled by driver. + */ + protected int driversNodesCnt; + + /** {@inheritDoc} */ + @Override public void setUp(BenchmarkConfiguration cfg) throws Exception { + super.setUp(cfg); + + memberId = cfg.memberId(); + + if (memberId < 0) + throw new IllegalStateException("Member id should be initialized with non-negative value"); + + // We assume there is no client nodes in the cluster except clients that are yardstick drivers. + driversNodesCnt = ignite().cluster().forClients().nodes().size(); + + IgniteCountDownLatch dataIsReady = ignite().countDownLatch("fillDataLatch", 1, true, true); + + try { + if (memberId == 0) { + fillData(cfg, (IgniteEx)ignite(), args.range(), args.atomicMode()); + + dataIsReady.countDown(); + } + else { + println(cfg, "No need to upload data for memberId=" + memberId + ". Just waiting"); + + dataIsReady.await(DATA_WAIT_TIMEOUT_MIN, TimeUnit.MINUTES); + + println(cfg, "Data is ready."); + } + + } + catch (Throwable th) { + dataIsReady.countDownAll(); + + throw new RuntimeException("Fill Data failed.", th); + } + + // Workaround for "Table TEST_LONG not found" on sql update. + execute(new SqlFieldsQuery("SELECT COUNT(*) FROM test_long")); + } + + /** + * Execute specified query using started driver node. + * Returns result using {@link QueryCursor#getAll()}. + * + * @param qry sql query to execute. + */ + protected List> execute(SqlFieldsQuery qry) { + return ((IgniteEx)ignite()) + .context() + .query() + .querySqlFields(qry, false) + .getAll(); + } +} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccProcessorBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccProcessorBenchmark.java new file mode 100644 index 0000000000000..543e7549fcee4 --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccProcessorBenchmark.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.yardstick.jdbc.mvcc; + +import java.util.Map; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.yardstick.jdbc.DisjointRangeGenerator; +import org.yardstickframework.BenchmarkConfiguration; + +/** + * Benchmark app that creates load on Mvcc Processor. + * Should be run in many threads on many hosts against one single server node. + */ +public class MvccProcessorBenchmark extends AbstractDistributedMvccBenchmark { + /** Generates id, that are disjoint only among threads running current host. */ + private DisjointRangeGenerator locIdGen; + + /** Offset for current host ids range, to make it disjoint among all the other host id ranges. */ + private int idOffset; + + /** {@inheritDoc} */ + @Override public void setUp(BenchmarkConfiguration cfg) throws Exception { + super.setUp(cfg); + + int locIdRangeWidth = args.range() / driversNodesCnt; + + locIdGen = new DisjointRangeGenerator(cfg.threads(), locIdRangeWidth, args.sqlRange()); + + idOffset = locIdRangeWidth * memberId; + } + + /** + * Performs sql updates on the key sets that are disjoint among all the threads on all the hosts. + */ + @Override public boolean test(Map ctx) throws Exception { + long locStart = locIdGen.nextRangeStartId(); + + long start = idOffset + locStart; + + long end = idOffset + locIdGen.endRangeId(locStart); + + execute(new SqlFieldsQuery(UPDATE_QRY).setArgs(start, end)); + + return true; + } +} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccUpdateContentionBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccUpdateContentionBenchmark.java new file mode 100644 index 0000000000000..09179dbf69a21 --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/mvcc/MvccUpdateContentionBenchmark.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.yardstick.jdbc.mvcc; + +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.processors.query.IgniteSQLException; + +import static org.yardstickframework.BenchmarkUtils.println; + +/** + * Benchmark app that eliminates update contention in mvcc mode. + * Designed to be ran in many threads on many hosts. + */ +public class MvccUpdateContentionBenchmark extends AbstractDistributedMvccBenchmark { + /** Expected expception message in mvcc on mode on update fail. */ + private static final String MVCC_EXC_MSG = "Mvcc version mismatch."; + + /** Expected exception message in mvcc off mode on update fail. */ + private static final String NO_MVCC_EXC_MSG_PREFIX = + "Failed to UPDATE some keys because they had been modified concurrently"; + + /** Counter of failed updates. */ + private final AtomicLong failsCnt = new AtomicLong(); + + /** {@inheritDoc} */ + @Override public boolean test(Map ctx) throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + long start = rnd.nextLong(args.mvccContentionRange() - (args.sqlRange() - 1)) + 1; + + long end = start + (args.sqlRange() - 1); + + try { + execute(new SqlFieldsQuery(UPDATE_QRY).setArgs(start, end)); + } + catch (IgniteSQLException exc) { + if ((args.mvccEnabled() && !exc.getMessage().equals(MVCC_EXC_MSG)) || + (!args.mvccEnabled() && !exc.getMessage().startsWith(NO_MVCC_EXC_MSG_PREFIX))) + throw new RuntimeException("Exception with unexpected message is thrown.", exc); + + failsCnt.incrementAndGet(); + } + catch (Exception e) { + throw new RuntimeException("Could not perform update.", e); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public void tearDown() throws Exception { + try { + super.tearDown(); + } + finally { + println("Update contention count : " + failsCnt.get()); + } + } +} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java index de1ad28be2e75..01eb6d46c643e 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractNativeBenchmark.java @@ -20,6 +20,7 @@ import java.util.Map; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.yardstick.IgniteAbstractBenchmark; import org.apache.ignite.yardstick.upload.model.Values10; import org.yardstickframework.BenchmarkConfiguration; @@ -58,7 +59,7 @@ public abstract class AbstractNativeBenchmark extends IgniteAbstractBenchmark { BenchmarkUtils.println(cfg, "Starting custom warmup."); String warmupCacheName = cacheName + "Warmup"; - try (IgniteCache warmupCache = ignite().createCache(warmupCacheName)) { + try (IgniteCache warmupCache = createCache(warmupCacheName)) { upload(warmupCacheName, warmupRowsCnt); } finally { @@ -68,7 +69,16 @@ public abstract class AbstractNativeBenchmark extends IgniteAbstractBenchmark { BenchmarkUtils.println(cfg, "Custom warmup finished."); // cache for benchmarked action - cache = ignite().createCache(cacheName); + cache = createCache(cacheName); + } + + private IgniteCache createCache(String name) { + CacheConfiguration cfg = new CacheConfiguration<>(name); + + if (args.atomicMode() != null) + cfg.setAtomicityMode(args.atomicMode()); + + return ignite().createCache(cfg); } /** {@inheritDoc} */ diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java index 20dfdd50816ae..6d9c84d47cc70 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/AbstractUploadBenchmark.java @@ -41,7 +41,7 @@ public abstract class AbstractUploadBenchmark extends AbstractJdbcBenchmark { long warmupRowsCnt; /** Factory that hides all the test data details. */ - protected QueryFactory queries = new QueryFactory(); + protected QueryFactory queries; /** {@inheritDoc} */ @Override public final void setUp(BenchmarkConfiguration cfg) throws Exception { @@ -93,7 +93,9 @@ protected void init() { /** * Creates empty table. */ - @Override protected void setupData() throws Exception{ + @Override protected void setupData() throws Exception { + queries = new QueryFactory(args.atomicMode()); + dropAndCreate(); } @@ -131,6 +133,9 @@ protected void init() { */ private void dropAndCreate() throws SQLException { executeUpdate(QueryFactory.DROP_TABLE_IF_EXISTS); + + BenchmarkUtils.println(cfg, "Creating table with schema: " + queries.createTable()); + executeUpdate(queries.createTable()); } diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java index 3ff4cb4166cc8..6c98d1aa35479 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/model/QueryFactory.java @@ -20,6 +20,7 @@ import java.sql.PreparedStatement; import java.sql.SQLException; import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.yardstick.upload.StreamerParams; import org.jetbrains.annotations.Nullable; @@ -47,16 +48,21 @@ public class QueryFactory { /** Number of "values" fields in the test table (any field except primary key). */ private int valFieldsCnt = 10; - /** Create table with long primary key and number of long and varchar fields */ - private String createTable = newCreateTableQuery(); - /** Parametrised query to insert new row. */ private String insert = newInsertQuery(); + /** Atomicity mode of test table's cache. */ + private CacheAtomicityMode tabAtomicMode; + + /** */ + public QueryFactory(CacheAtomicityMode tabAtomicMode) { + this.tabAtomicMode = tabAtomicMode; + } + /** - * See {@link #createTable}. + * Create table with long primary key and number of long and varchar fields */ - private String newCreateTableQuery() { + public String createTable() { StringBuilder create = new StringBuilder("CREATE TABLE test_upload (id LONG PRIMARY KEY"); for (int vi = 1; vi <= valFieldsCnt; vi++) { @@ -69,7 +75,12 @@ private String newCreateTableQuery() { } - create.append(");"); + create.append(')'); + + if (tabAtomicMode != null) + create.append(" WITH \"ATOMICITY=").append(tabAtomicMode.name()).append('\"'); + + create.append(';'); return create.toString(); } @@ -86,13 +97,6 @@ private String newInsertQuery() { return insert.toString(); } - /** - * See {@link #createTable}. - */ - public String createTable() { - return createTable; - } - /** * See {@link #insert}. */ From eec22c55249669091edf404f126181a350197ada Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Thu, 30 Aug 2018 15:02:50 +0300 Subject: [PATCH 20/95] IGNITE-6552 Added ability to set WAL history size in bytes - Fixes #3559. --- .../apache/ignite/IgniteSystemProperties.java | 11 + .../DataStorageConfiguration.java | 42 +++ .../wal/IgniteWriteAheadLogManager.java | 5 + .../GridCacheDatabaseSharedManager.java | 16 +- .../IgniteCacheDatabaseSharedManager.java | 34 +++ .../checkpoint/CheckpointHistory.java | 102 +++++-- .../cache/persistence/wal/FileDescriptor.java | 136 +++++++++ .../wal/FileWriteAheadLogManager.java | 185 ++++++------ .../FsyncModeFileWriteAheadLogManager.java | 214 ++++++-------- .../SingleSegmentLogicalRecordsIterator.java | 4 +- .../wal/reader/IgniteWalIteratorFactory.java | 2 +- .../reader/StandaloneWalRecordsIterator.java | 2 +- .../utils/PlatformConfigurationUtils.java | 2 + ...a => IgnitePdsReserveWalSegmentsTest.java} | 88 +++--- ...ReachedDuringIterationOverArchiveTest.java | 2 +- .../persistence/db/wal/WalCompactionTest.java | 8 +- .../wal/WalDeletionArchiveAbstractTest.java | 277 ++++++++++++++++++ .../db/wal/WalDeletionArchiveFsyncTest.java | 31 ++ .../db/wal/WalDeletionArchiveLogOnlyTest.java | 31 ++ .../wal/WalRecoveryTxLogicalRecordsTest.java | 44 --- .../db/wal/reader/MockWalIteratorFactory.java | 1 + .../persistence/pagemem/NoOpWALManager.java | 6 + .../testsuites/IgnitePdsTestSuite2.java | 14 +- .../DataStorageConfigurationParityTest.cs | 3 +- .../Configuration/DataStorageConfiguration.cs | 14 + .../IgniteConfigurationSection.xsd | 5 + 26 files changed, 918 insertions(+), 361 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java rename modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/{IgnitePdsUnusedWalSegmentsTest.java => IgnitePdsReserveWalSegmentsTest.java} (65%) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveFsyncTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveLogOnlyTest.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 78842d64aaca0..00ab49a1914da 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -874,6 +874,17 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_LOADED_PAGES_BACKWARD_SHIFT_MAP = "IGNITE_LOADED_PAGES_BACKWARD_SHIFT_MAP"; + /** + * Property for setup percentage of archive size for checkpoint trigger. Default value is 0.25 + */ + public static final String IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE = "IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE"; + + /** + * Property for setup percentage of WAL archive size to calculate threshold since which removing of old archive should be started. + * Default value is 0.5 + */ + public static final String IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = "IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE"; + /** * Whenever read load balancing is enabled, that means 'get' requests will be distributed between primary and backup * nodes if it is possible and {@link CacheConfiguration#readFromBackup} is {@code true}. diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java index d8619aa4bfca9..e7e8ca3c098d2 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java @@ -114,6 +114,9 @@ public class DataStorageConfiguration implements Serializable { /** Default number of checkpoints to be kept in WAL after checkpoint is finished */ public static final int DFLT_WAL_HISTORY_SIZE = 20; + /** Default max size of WAL archive files, in bytes */ + public static final long DFLT_WAL_ARCHIVE_MAX_SIZE = 1024 * 1024 * 1024; + /** */ public static final int DFLT_WAL_SEGMENTS = 10; @@ -190,6 +193,9 @@ public class DataStorageConfiguration implements Serializable { /** Number of checkpoints to keep */ private int walHistSize = DFLT_WAL_HISTORY_SIZE; + /** Maximum size of wal archive folder, in bytes */ + private long maxWalArchiveSize = DFLT_WAL_ARCHIVE_MAX_SIZE; + /** Number of work WAL segments. */ private int walSegments = DFLT_WAL_SEGMENTS; @@ -487,7 +493,10 @@ public DataStorageConfiguration setLockWaitTime(long lockWaitTime) { * Gets a total number of checkpoints to keep in the WAL history. * * @return Number of checkpoints to keep in WAL after a checkpoint is finished. + * @deprecated Instead of walHistorySize use maxWalArchiveSize for manage of archive size. + * @see DataStorageConfiguration#getMaxWalArchiveSize() */ + @Deprecated public int getWalHistorySize() { return walHistSize <= 0 ? DFLT_WAL_HISTORY_SIZE : walHistSize; } @@ -497,13 +506,46 @@ public int getWalHistorySize() { * * @param walHistSize Number of checkpoints to keep after a checkpoint is finished. * @return {@code this} for chaining. + * @deprecated Instead of walHistorySize use maxWalArchiveSize for manage of archive size. + * @see DataStorageConfiguration#setMaxWalArchiveSize(long) */ + @Deprecated public DataStorageConfiguration setWalHistorySize(int walHistSize) { this.walHistSize = walHistSize; return this; } + /** + * If WalHistorySize was set by user will use this parameter for compatibility. + * + * @return {@code true} if use WalHistorySize for compatibility. + */ + public boolean isWalHistorySizeParameterUsed() { + return getWalHistorySize() != DFLT_WAL_HISTORY_SIZE && getWalHistorySize() != Integer.MAX_VALUE; + } + + /** + * Gets a max allowed size of WAL archives. In bytes. + * + * @return max size of WAL archive directory. + */ + public long getMaxWalArchiveSize() { + return maxWalArchiveSize <= 0 ? DFLT_WAL_ARCHIVE_MAX_SIZE : maxWalArchiveSize; + } + + /** + * Sets a max allowed size of WAL archives. In bytes + * + * @param walArchiveMaxSize max size of WAL archive directory. + * @return {@code this} for chaining. + */ + public DataStorageConfiguration setMaxWalArchiveSize(long walArchiveMaxSize) { + this.maxWalArchiveSize = walArchiveMaxSize; + + return this; + } + /** * Gets a number of WAL segments to work with. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java index b5282ff7ad9a4..12fd3e94bd0fe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java @@ -134,6 +134,11 @@ public interface IgniteWriteAheadLogManager extends GridCacheSharedManager, Igni */ public long lastCompactedSegment(); + /** + * @return Max allowed index of archived segment to delete or -1 if it does not exist. + */ + public long maxArchivedSegmentToDelete(); + /** * Checks if WAL segment is under lock or reserved * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 7d91645b57e64..7393c017a5149 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -167,6 +167,7 @@ import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CHECKPOINT_RECORD; +import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_HISTORY_SIZE; import static org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID; /** @@ -360,7 +361,9 @@ public GridCacheDatabaseSharedManager(GridKernalContext ctx) { checkpointFreq = persistenceCfg.getCheckpointFrequency(); - truncateWalOnCpFinish = persistenceCfg.getWalHistorySize() != Integer.MAX_VALUE; + truncateWalOnCpFinish = persistenceCfg.isWalHistorySizeParameterUsed() + ? persistenceCfg.getWalHistorySize() != Integer.MAX_VALUE + : persistenceCfg.getMaxWalArchiveSize() != Long.MAX_VALUE; lockWaitTime = persistenceCfg.getLockWaitTime(); @@ -497,7 +500,7 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu /** * Cleanup checkpoint directory from all temporary files {@link #FILE_TMP_SUFFIX}. */ - public void cleanupTempCheckpointDirectory() throws IgniteCheckedException { + @Override public void cleanupTempCheckpointDirectory() throws IgniteCheckedException { try { try (DirectoryStream files = Files.newDirectoryStream( cpDir.toPath(), @@ -515,7 +518,7 @@ public void cleanupTempCheckpointDirectory() throws IgniteCheckedException { /** * Cleanup checkpoint directory. */ - public void cleanupCheckpointDirectory() throws IgniteCheckedException { + @Override public void cleanupCheckpointDirectory() throws IgniteCheckedException { try { try (DirectoryStream files = Files.newDirectoryStream(cpDir.toPath())) { for (Path path : files) @@ -1780,6 +1783,13 @@ private Map> partitionsApplicableForWalRebalance() { return cp.wakeupForCheckpoint(0, reason); } + /** {@inheritDoc} */ + @Override public WALPointer lastCheckpointMarkWalPointer() { + CheckpointEntry lastCheckpointEntry = cpHistory == null ? null : cpHistory.lastCheckpoint(); + + return lastCheckpointEntry == null ? null : lastCheckpointEntry.checkpointMark(); + } + /** * @return Checkpoint directory. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java index 3ade2657c4995..737b02aa9f06c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java @@ -42,6 +42,7 @@ import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl; +import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; @@ -69,6 +70,8 @@ import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME; import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE; +import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_ARCHIVE_MAX_SIZE; +import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_HISTORY_SIZE; /** * @@ -388,6 +391,30 @@ private void validateConfiguration(DataStorageConfiguration memCfg) throws Ignit } checkDataRegionConfiguration(memCfg, regNames, memCfg.getDefaultDataRegionConfiguration()); + + checkWalArchiveSizeConfiguration(memCfg); + } + + /** + * Check wal archive size configuration for correctness. + * + * @param memCfg durable memory configuration for an Apache Ignite node. + */ + private void checkWalArchiveSizeConfiguration(DataStorageConfiguration memCfg) throws IgniteCheckedException { + if (memCfg.getWalHistorySize() == DFLT_WAL_HISTORY_SIZE || memCfg.getWalHistorySize() == Integer.MAX_VALUE) + LT.warn(log, "DataRegionConfiguration.maxWalArchiveSize instead DataRegionConfiguration.walHistorySize " + + "would be used for removing old archive wal files"); + else if(memCfg.getMaxWalArchiveSize() == DFLT_WAL_ARCHIVE_MAX_SIZE) + LT.warn(log, "walHistorySize was deprecated. maxWalArchiveSize should be used instead"); + else + throw new IgniteCheckedException("Should be used only one of wal history size or max wal archive size." + + "(use DataRegionConfiguration.maxWalArchiveSize because DataRegionConfiguration.walHistorySize was deprecated)" + ); + + if(memCfg.getMaxWalArchiveSize() < memCfg.getWalSegmentSize()) + throw new IgniteCheckedException( + "DataRegionConfiguration.maxWalArchiveSize should be greater than DataRegionConfiguration.walSegmentSize" + ); } /** @@ -747,6 +774,13 @@ public void cleanupTempCheckpointDirectory() throws IgniteCheckedException{ return null; } + /** + * @return Last checkpoint mark WAL pointer. + */ + public WALPointer lastCheckpointMarkWalPointer() { + return null; + } + /** * Allows to wait checkpoint finished. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java index 3fb845754fdcc..e64a53f477974 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.Checkpoint; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -64,6 +65,9 @@ public class CheckpointHistory { /** The maximal number of checkpoints hold in memory. */ private final int maxCpHistMemSize; + /** If WalHistorySize was setted by user will use old way for removing checkpoints. */ + private final boolean isWalHistorySizeParameterEnabled; + /** * Constructor. * @@ -77,6 +81,8 @@ public CheckpointHistory(GridKernalContext ctx) { maxCpHistMemSize = Math.min(dsCfg.getWalHistorySize(), IgniteSystemProperties.getInteger(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, 100)); + + isWalHistorySizeParameterEnabled = dsCfg.isWalHistorySizeParameterUsed(); } /** @@ -198,36 +204,98 @@ public List onWalTruncated(WALPointer ptr) { * * @return List of checkpoints removed from history. */ - public List onCheckpointFinished(GridCacheDatabaseSharedManager.Checkpoint chp, boolean truncateWal) { - List rmv = new ArrayList<>(); - + public List onCheckpointFinished(Checkpoint chp, boolean truncateWal) { chp.walSegsCoveredRange(calculateWalSegmentsCovered()); + WALPointer checkpointMarkUntilDel = isWalHistorySizeParameterEnabled //check for compatibility mode. + ? checkpointMarkUntilDeleteByMemorySize() + : newerPointer(checkpointMarkUntilDeleteByMemorySize(), checkpointMarkUntilDeleteByArchiveSize()); + + if (checkpointMarkUntilDel == null) + return Collections.emptyList(); + + List deletedCheckpoints = onWalTruncated(checkpointMarkUntilDel); + int deleted = 0; - while (histMap.size() > maxCpHistMemSize) { - Map.Entry entry = histMap.firstEntry(); + if (truncateWal) + deleted += cctx.wal().truncate(null, firstCheckpointPointer()); - CheckpointEntry cpEntry = entry.getValue(); + chp.walFilesDeleted(deleted); - if (cctx.wal().reserved(cpEntry.checkpointMark())) { - U.warn(log, "Could not clear historyMap due to WAL reservation on cpEntry " + cpEntry.checkpointId() + - ", history map size is " + histMap.size()); + return deletedCheckpoints; + } - break; - } + /** + * @param firstPointer One of pointers to choose the newest. + * @param secondPointer One of pointers to choose the newest. + * @return The newest pointer from input ones. + */ + private FileWALPointer newerPointer(WALPointer firstPointer, WALPointer secondPointer) { + FileWALPointer first = (FileWALPointer)firstPointer; + FileWALPointer second = (FileWALPointer)secondPointer; + + if (firstPointer == null) + return second; + + if (secondPointer == null) + return first; - if (truncateWal) - deleted += cctx.wal().truncate(null, cpEntry.checkpointMark()); + return first.index() > second.index() ? first : second; + } - histMap.remove(entry.getKey()); + /** + * Calculate mark until delete by maximum checkpoint history memory size. + * + * @return Checkpoint mark until which checkpoints can be deleted(not including this pointer). + */ + private WALPointer checkpointMarkUntilDeleteByMemorySize() { + if (histMap.size() <= maxCpHistMemSize) + return null; - rmv.add(cpEntry); + int calculatedCpHistSize = maxCpHistMemSize; + + for (Map.Entry entry : histMap.entrySet()) { + if (histMap.size() <= calculatedCpHistSize++) + return entry.getValue().checkpointMark(); } - chp.walFilesDeleted(deleted); + return lastCheckpoint().checkpointMark(); + } + + /** + * Calculate mark until delete by maximum allowed archive size. + * + * @return Checkpoint mark until which checkpoints can be deleted(not including this pointer). + */ + @Nullable private WALPointer checkpointMarkUntilDeleteByArchiveSize() { + long absFileIdxToDel = cctx.wal().maxArchivedSegmentToDelete(); + + if (absFileIdxToDel < 0) + return null; + + long fileUntilDel = absFileIdxToDel + 1; - return rmv; + long checkpointFileIdx = absFileIdx(lastCheckpoint()); + + for (CheckpointEntry cpEntry : histMap.values()) { + long currFileIdx = absFileIdx(cpEntry); + + if (checkpointFileIdx <= currFileIdx || fileUntilDel <= currFileIdx) + return cpEntry.checkpointMark(); + } + + return lastCheckpoint().checkpointMark(); + } + + /** + * Retrieve absolute file index by checkpoint entry. + * + * @param pointer checkpoint entry for which need to calculate absolute file index. + * @return absolute file index for given checkpoint entry. + */ + private long absFileIdx(CheckpointEntry pointer) { + return ((FileWALPointer)pointer.checkpointMark()).index(); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java new file mode 100644 index 0000000000000..39052cc92fcd1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java @@ -0,0 +1,136 @@ +/* + * 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.persistence.wal; + +import org.apache.ignite.internal.util.typedef.internal.SB; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.File; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * WAL file descriptor. + */ +public class FileDescriptor implements Comparable, AbstractWalRecordsIterator.AbstractFileDescriptor { + + /** file extension of WAL segment. */ + private static final String WAL_SEGMENT_FILE_EXT = ".wal"; + + /** Length of WAL segment file name. */ + private static final int WAL_SEGMENT_FILE_NAME_LENGTH = 16; + + /** File represented by this class. */ + protected final File file; + + /** Absolute WAL segment file index. */ + protected final long idx; + + /** + * Creates file descriptor. Index is restored from file name. + * + * @param file WAL segment file. + */ + public FileDescriptor(@NotNull File file) { + this(file, null); + } + + /** + * @param file WAL segment file. + * @param idx Absolute WAL segment file index. For null value index is restored from file name. + */ + public FileDescriptor(@NotNull File file, @Nullable Long idx) { + this.file = file; + + String fileName = file.getName(); + + assert fileName.contains(WAL_SEGMENT_FILE_EXT); + + this.idx = idx == null ? Long.parseLong(fileName.substring(0, WAL_SEGMENT_FILE_NAME_LENGTH)) : idx; + } + + /** + * @param segment Segment index. + * @return Segment file name. + */ + public static String fileName(long segment) { + SB b = new SB(); + + String segmentStr = Long.toString(segment); + + for (int i = segmentStr.length(); i < WAL_SEGMENT_FILE_NAME_LENGTH; i++) + b.a('0'); + + b.a(segmentStr).a(WAL_SEGMENT_FILE_EXT); + + return b.toString(); + } + + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull FileDescriptor o) { + return Long.compare(idx, o.idx); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (!(o instanceof FileDescriptor)) + return false; + + FileDescriptor that = (FileDescriptor)o; + + return idx == that.idx; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return (int)(idx ^ (idx >>> 32)); + } + + /** + * @return Absolute WAL segment file index + */ + public long getIdx() { + return idx; + } + + /** + * @return absolute pathname string of this file descriptor pathname. + */ + public String getAbsolutePath() { + return file.getAbsolutePath(); + } + + /** {@inheritDoc} */ + @Override public boolean isCompressed() { + return file.getName().endsWith(".zip"); + } + + /** {@inheritDoc} */ + @Override public File file() { + return file; + } + + /** {@inheritDoc} */ + @Override public long idx() { + return idx; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java index 3fa499b4cdb07..61def1e7bd41e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java @@ -59,6 +59,7 @@ import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Pattern; +import java.util.stream.Stream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; @@ -110,7 +111,6 @@ import org.apache.ignite.internal.util.typedef.CIX1; import org.apache.ignite.internal.util.typedef.CO; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.lang.IgniteBiTuple; @@ -124,6 +124,8 @@ import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.READ; import static java.nio.file.StandardOpenOption.WRITE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_MMAP; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SEGMENT_SYNC_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; @@ -170,9 +172,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** */ private static final FileDescriptor[] EMPTY_DESCRIPTORS = new FileDescriptor[0]; - /** WAL segment file extension. */ - private static final String WAL_SEGMENT_FILE_EXT = ".wal"; - /** */ private static final byte[] FILL_BUF = new byte[1024 * 1024]; @@ -237,6 +236,19 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl private static final AtomicLongFieldUpdater WRITTEN_UPD = AtomicLongFieldUpdater.newUpdater(FileWriteHandle.class, "written"); + /** + * Percentage of archive size for checkpoint trigger. Need for calculate max size of WAL after last checkpoint. + * Checkpoint should be triggered when max size of WAL after last checkpoint more than maxWallArchiveSize * thisValue + */ + private static final double CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE = + IgniteSystemProperties.getDouble(IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE, 0.25); + + /** + * Percentage of WAL archive size to calculate threshold since which removing of old archive should be started. + */ + private static final double THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = + IgniteSystemProperties.getDouble(IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE, 0.5); + /** Interrupted flag. */ private final ThreadLocal interrupted = new ThreadLocal() { @Override protected Boolean initialValue() { @@ -250,6 +262,15 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** WAL segment size in bytes. . This is maximum value, actual segments may be shorter. */ private final long maxWalSegmentSize; + /** + * Maximum number of allowed segments without checkpoint. If we have their more checkpoint should be triggered. + * It is simple way to calculate WAL size without checkpoint instead fair WAL size calculating. + */ + private final long maxSegCountWithoutCheckpoint; + + /** Size of wal archive since which removing of old archive should be started */ + private final long allowedThresholdWalArchiveSize; + /** */ private final WALMode mode; @@ -304,8 +325,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl AtomicReferenceFieldUpdater.newUpdater(FileWriteAheadLogManager.class, FileWriteHandle.class, "currHnd"); /** - * File archiver moves segments from work directory to archive. Locked segments may be kept not moved until - * release. For mode archive and work folders set to equal value, archiver is not created. + * File archiver moves segments from work directory to archive. Locked segments may be kept not moved until release. + * For mode archive and work folders set to equal value, archiver is not created. */ @Nullable private volatile FileArchiver archiver; @@ -379,6 +400,12 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { alwaysWriteFullPages = dsCfg.isAlwaysWriteFullPages(); ioFactory = new RandomAccessFileIOFactory(); walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity(); + + maxSegCountWithoutCheckpoint = + (long)((dsCfg.getMaxWalArchiveSize() * CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE) / dsCfg.getWalSegmentSize()); + + allowedThresholdWalArchiveSize = (long)(dsCfg.getMaxWalArchiveSize() * THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE); + evt = ctx.event(); failureProcessor = ctx.failure(); } @@ -760,7 +787,7 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { rec.size(serializer.size(rec)); while (true) { - if (rec.rollOver()){ + if (rec.rollOver()) { assert cctx.database().checkpointLockIsHeldByThread(); long idx = currWrHandle.idx; @@ -1141,6 +1168,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I next.writeHeader(); + if (next.idx - lashCheckpointFileIdx() >= maxSegCountWithoutCheckpoint) + cctx.database().forceCheckpoint("too big size of WAL without checkpoint"); + boolean swapped = CURR_HND_UPD.compareAndSet(this, hnd, next); assert swapped : "Concurrent updates on rollover are not allowed"; @@ -1157,6 +1187,15 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I return currentHandle(); } + /** + * Give last checkpoint file idx. + */ + private long lashCheckpointFileIdx() { + WALPointer lastCheckpointMark = cctx.database().lastCheckpointMarkWalPointer(); + + return lastCheckpointMark == null ? 0 : ((FileWALPointer)lastCheckpointMark).index(); + } + /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. @@ -1297,7 +1336,6 @@ private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws IgniteCh serializer, rbuf); - if (interrupted) Thread.currentThread().interrupt(); @@ -1367,7 +1405,7 @@ private void checkOrPrepareFiles() throws StorageException { File[] allFiles = walWorkDir.listFiles(WAL_SEGMENT_FILE_FILTER); - if(isArchiverEnabled()) + if (isArchiverEnabled()) if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) throw new StorageException("Failed to initialize wal (work directory contains " + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); @@ -1498,6 +1536,39 @@ private File pollNextFile(long curIdx) throws StorageException { return new File(walWorkDir, FileDescriptor.fileName(segmentIdx)); } + /** + * Files from archive WAL directory. + */ + private FileDescriptor[] walArchiveFiles() { + return scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); + } + + /** {@inheritDoc} */ + @Override public long maxArchivedSegmentToDelete() { + //When maxWalArchiveSize==MAX_VALUE deleting files is not permit. + if (dsCfg.getMaxWalArchiveSize() == Long.MAX_VALUE) + return -1; + + FileDescriptor[] archivedFiles = walArchiveFiles(); + + Long totalArchiveSize = Stream.of(archivedFiles) + .map(desc -> desc.file().length()) + .reduce(0L, Long::sum); + + if (archivedFiles.length == 0 || totalArchiveSize < allowedThresholdWalArchiveSize) + return -1; + + long sizeOfOldestArchivedFiles = 0; + + for (FileDescriptor desc : archivedFiles) { + sizeOfOldestArchivedFiles += desc.file().length(); + + if (totalArchiveSize - sizeOfOldestArchivedFiles < allowedThresholdWalArchiveSize) + return desc.getIdx(); + } + + return archivedFiles[archivedFiles.length - 1].getIdx(); + } /** * @return Sorted WAL files descriptors. @@ -2366,98 +2437,6 @@ else if (create) return buf; } - /** - * WAL file descriptor. - */ - public static class FileDescriptor implements - Comparable, AbstractFileDescriptor { - /** */ - protected final File file; - - /** Absolute WAL segment file index */ - protected final long idx; - - /** - * Creates file descriptor. Index is restored from file name - * - * @param file WAL segment file. - */ - public FileDescriptor(@NotNull File file) { - this(file, null); - } - - /** - * @param file WAL segment file. - * @param idx Absolute WAL segment file index. For null value index is restored from file name - */ - public FileDescriptor(@NotNull File file, @Nullable Long idx) { - this.file = file; - - String fileName = file.getName(); - - assert fileName.contains(WAL_SEGMENT_FILE_EXT); - - this.idx = idx == null ? Long.parseLong(fileName.substring(0, 16)) : idx; - } - - /** - * @param segment Segment index. - * @return Segment file name. - */ - public static String fileName(long segment) { - SB b = new SB(); - - String segmentStr = Long.toString(segment); - - for (int i = segmentStr.length(); i < 16; i++) - b.a('0'); - - b.a(segmentStr).a(WAL_SEGMENT_FILE_EXT); - - return b.toString(); - } - - /** {@inheritDoc} */ - @Override public int compareTo(@NotNull FileDescriptor o) { - return Long.compare(idx, o.idx); - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - - if (!(o instanceof FileDescriptor)) - return false; - - FileDescriptor that = (FileDescriptor)o; - - return idx == that.idx; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return (int)(idx ^ (idx >>> 32)); - } - - /** - * @return True if segment is ZIP compressed. - */ - @Override public boolean isCompressed() { - return file.getName().endsWith(".zip"); - } - - /** {@inheritDoc} */ - @Override public File file() { - return file; - } - - /** {@inheritDoc} */ - @Override public long idx() { - return idx; - } - } - /** * */ @@ -2615,7 +2594,6 @@ private FileWriteHandle( this.buf = buf; } - /** * Write serializer version to current handle. */ @@ -3040,7 +3018,8 @@ private static class RecordsIterator extends AbstractWalRecordsIterator { * @param serializerFactory Serializer factory. * @param archiver File Archiver. * @param decompressor Decompressor. - *@param log Logger @throws IgniteCheckedException If failed to initialize WAL segment. + * @param log Logger + * @throws IgniteCheckedException If failed to initialize WAL segment. */ private RecordsIterator( GridCacheSharedContext cctx, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java index 4797f81ea0480..91d3218572e64 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java @@ -55,6 +55,7 @@ import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Pattern; +import java.util.stream.Stream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; @@ -103,7 +104,6 @@ import org.apache.ignite.internal.util.typedef.CO; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.lang.IgniteBiTuple; @@ -117,6 +117,8 @@ import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.READ; import static java.nio.file.StandardOpenOption.WRITE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_COMPACTED; import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; @@ -130,9 +132,6 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda /** */ public static final FileDescriptor[] EMPTY_DESCRIPTORS = new FileDescriptor[0]; - /** */ - public static final String WAL_SEGMENT_FILE_EXT = ".wal"; - /** */ private static final byte[] FILL_BUF = new byte[1024 * 1024]; @@ -187,12 +186,34 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda /** Latest serializer version to use. */ private static final int LATEST_SERIALIZER_VERSION = 2; + /** + * Percentage of archive size for checkpoint trigger. Need for calculate max size of WAL after last checkpoint. + * Checkpoint should be triggered when max size of WAL after last checkpoint more than maxWallArchiveSize * thisValue + */ + private static final double CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE = + IgniteSystemProperties.getDouble(IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE, 0.25); + + /** + * Percentage of WAL archive size to calculate threshold since which removing of old archive should be started. + */ + private static final double THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = + IgniteSystemProperties.getDouble(IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE, 0.5); + /** */ private final boolean alwaysWriteFullPages; /** WAL segment size in bytes */ private final long maxWalSegmentSize; + /** + * Maximum number of allowed segments without checkpoint. If we have their more checkpoint should be triggered. + * It is simple way to calculate wal size without checkpoint instead fair wal size calculating. + */ + private final long maxSegCountWithoutCheckpoint; + + /** Size of wal archive since which removing of old archive should be started */ + private final long allowedThresholdWalArchiveSize; + /** */ private final WALMode mode; @@ -321,6 +342,11 @@ public FsyncModeFileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity(); evt = ctx.event(); + maxSegCountWithoutCheckpoint = + (long)((dsCfg.getMaxWalArchiveSize() * CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE) / dsCfg.getWalSegmentSize()); + + allowedThresholdWalArchiveSize = (long)(dsCfg.getMaxWalArchiveSize() * THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE); + assert mode == WALMode.FSYNC : dsCfg; } @@ -589,7 +615,7 @@ public Collection getAndReserveWalFiles(FileWALPointer low, FileWALPointer List res = new ArrayList<>(); for (long i = low.index(); i < high.index(); i++) { - String segmentName = FileWriteAheadLogManager.FileDescriptor.fileName(i); + String segmentName = FileDescriptor.fileName(i); File file = new File(walArchiveDir, segmentName); File fileZip = new File(walArchiveDir, segmentName + ".zip"); @@ -869,6 +895,40 @@ private boolean hasIndex(long absIdx) { return res >= 0 ? res : 0; } + /** + * Files from archive WAL directory. + */ + private FileDescriptor[] walArchiveFiles() { + return scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); + } + + /** {@inheritDoc} */ + @Override public long maxArchivedSegmentToDelete() { + //When maxWalArchiveSize==MAX_VALUE deleting files is not permit. + if (dsCfg.getMaxWalArchiveSize() == Long.MAX_VALUE) + return -1; + + FileDescriptor[] archivedFiles = walArchiveFiles(); + + Long totalArchiveSize = Stream.of(archivedFiles) + .map(desc -> desc.file().length()) + .reduce(0L, Long::sum); + + if (archivedFiles.length == 0 || totalArchiveSize < allowedThresholdWalArchiveSize) + return -1; + + long sizeOfOldestArchivedFiles = 0; + + for (FileDescriptor desc : archivedFiles) { + sizeOfOldestArchivedFiles += desc.file().length(); + + if (totalArchiveSize - sizeOfOldestArchivedFiles < allowedThresholdWalArchiveSize) + return desc.getIdx(); + } + + return archivedFiles[archivedFiles.length - 1].getIdx(); + } + /** {@inheritDoc} */ @Override public long lastArchivedSegment() { return archiver.lastArchivedAbsoluteIndex(); @@ -1058,6 +1118,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws IgniteCheckedExcept FileWriteHandle next = initNextWriteHandle(cur.idx); + if (next.idx - lashCheckpointFileIdx() >= maxSegCountWithoutCheckpoint) + cctx.database().forceCheckpoint("too big size of WAL without checkpoint"); + boolean swapped = currentHndUpd.compareAndSet(this, hnd, next); assert swapped : "Concurrent updates on rollover are not allowed"; @@ -1074,6 +1137,15 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws IgniteCheckedExcept return currentHandle(); } + /** + * Give last checkpoint file idx + */ + private long lashCheckpointFileIdx() { + WALPointer lastCheckpointMark = cctx.database().lastCheckpointMarkWalPointer(); + + return lastCheckpointMark == null ? 0 : ((FileWALPointer)lastCheckpointMark).index(); + } + /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. @@ -1820,19 +1892,19 @@ private long tryReserveNextSegmentOrWait() throws InterruptedException, IgniteCh * Deletes raw WAL segments if they aren't locked and already have compressed copies of themselves. */ private void deleteObsoleteRawSegments() { - FsyncModeFileWriteAheadLogManager.FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); + FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); Set indices = new HashSet<>(); Set duplicateIndices = new HashSet<>(); - for (FsyncModeFileWriteAheadLogManager.FileDescriptor desc : descs) { + for (FileDescriptor desc : descs) { if (!indices.add(desc.idx)) duplicateIndices.add(desc.idx); } FileArchiver archiver0 = archiver; - for (FsyncModeFileWriteAheadLogManager.FileDescriptor desc : descs) { + for (FileDescriptor desc : descs) { if (desc.isCompressed()) continue; @@ -1862,11 +1934,11 @@ private void deleteObsoleteRawSegments() { if (currReservedSegment == -1) continue; - File tmpZip = new File(walArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(currReservedSegment) + ".zip" + ".tmp"); + File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip" + ".tmp"); - File zip = new File(walArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(currReservedSegment) + ".zip"); + File zip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip"); - File raw = new File(walArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(currReservedSegment)); + File raw = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment)); if (!Files.exists(raw.toPath())) throw new IgniteCheckedException("WAL archive segment is missing: " + raw); @@ -2193,126 +2265,6 @@ public static long writeSerializerVersion(FileIO io, long idx, int version, WALM return buf; } - /** - * WAL file descriptor. - */ - public static class FileDescriptor implements Comparable, AbstractWalRecordsIterator.AbstractFileDescriptor { - /** */ - protected final File file; - - /** Absolute WAL segment file index */ - protected final long idx; - - /** - * Creates file descriptor. Index is restored from file name - * - * @param file WAL segment file. - */ - public FileDescriptor(@NotNull File file) { - this(file, null); - } - - /** - * @param file WAL segment file. - * @param idx Absolute WAL segment file index. For null value index is restored from file name - */ - public FileDescriptor(@NotNull File file, @Nullable Long idx) { - this.file = file; - - String fileName = file.getName(); - - assert fileName.contains(WAL_SEGMENT_FILE_EXT); - - this.idx = idx == null ? Long.parseLong(fileName.substring(0, 16)) : idx; - } - - /** - * @param segment Segment index. - * @return Segment file name. - */ - public static String fileName(long segment) { - SB b = new SB(); - - String segmentStr = Long.toString(segment); - - for (int i = segmentStr.length(); i < 16; i++) - b.a('0'); - - b.a(segmentStr).a(WAL_SEGMENT_FILE_EXT); - - return b.toString(); - } - - /** - * @param segment Segment number as integer. - * @return Segment number as aligned string. - */ - private static String segmentNumber(long segment) { - SB b = new SB(); - - String segmentStr = Long.toString(segment); - - for (int i = segmentStr.length(); i < 16; i++) - b.a('0'); - - b.a(segmentStr); - - return b.toString(); - } - - /** {@inheritDoc} */ - @Override public int compareTo(FileDescriptor o) { - return Long.compare(idx, o.idx); - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - - if (!(o instanceof FileDescriptor)) - return false; - - FileDescriptor that = (FileDescriptor)o; - - return idx == that.idx; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return (int)(idx ^ (idx >>> 32)); - } - - /** - * @return Absolute WAL segment file index - */ - public long getIdx() { - return idx; - } - - /** - * @return absolute pathname string of this file descriptor pathname. - */ - public String getAbsolutePath() { - return file.getAbsolutePath(); - } - - /** {@inheritDoc} */ - @Override public boolean isCompressed() { - return file.getName().endsWith(".zip"); - } - - /** {@inheritDoc} */ - @Override public File file() { - return file; - } - - /** {@inheritDoc} */ - @Override public long idx() { - return idx; - } - } - /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java index f688bb4b5f783..a42eb89897407 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java @@ -100,8 +100,8 @@ private static RecordSerializerFactory initLogicalRecordsSerializerFactory(GridC else { segmentInitialized = true; - FileWriteAheadLogManager.FileDescriptor fd = new FileWriteAheadLogManager.FileDescriptor( - new File(archiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(curWalSegmIdx))); + FileDescriptor fd = new FileDescriptor( + new File(archiveDir, FileDescriptor.fileName(curWalSegmIdx))); try { return initReadHandle(fd, null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java index cda8d90174281..0e61faed7c5b4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java @@ -43,9 +43,9 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.UnzipFileIO; import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; -import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.FileDescriptor; 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.U; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java index e934f33bfd2bb..67297905c1a43 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java @@ -41,9 +41,9 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.UnzipFileIO; import org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; -import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.FileDescriptor; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.ReadFileHandle; import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; 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 79780f7739570..cd67f15246ca3 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 @@ -1820,6 +1820,7 @@ private static DataStorageConfiguration readDataStorageConfiguration(BinaryRawRe .setCheckpointWriteOrder(CheckpointWriteOrder.fromOrdinal(in.readInt())) .setWriteThrottlingEnabled(in.readBoolean()) .setWalCompactionEnabled(in.readBoolean()) + .setMaxWalArchiveSize(in.readLong()) .setSystemRegionInitialSize(in.readLong()) .setSystemRegionMaxSize(in.readLong()) .setPageSize(in.readInt()) @@ -1947,6 +1948,7 @@ private static void writeDataStorageConfiguration(BinaryRawWriter w, DataStorage w.writeInt(cfg.getCheckpointWriteOrder().ordinal()); w.writeBoolean(cfg.isWriteThrottlingEnabled()); w.writeBoolean(cfg.isWalCompactionEnabled()); + w.writeLong(cfg.getMaxWalArchiveSize()); w.writeLong(cfg.getSystemRegionInitialSize()); w.writeLong(cfg.getSystemRegionMaxSize()); w.writeInt(cfg.getPageSize()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsUnusedWalSegmentsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java similarity index 65% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsUnusedWalSegmentsTest.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java index 06a9ec271ef25..5885b7a5ed869 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsUnusedWalSegmentsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java @@ -40,7 +40,7 @@ /** * Test correctness of truncating unused WAL segments. */ -public class IgnitePdsUnusedWalSegmentsTest extends GridCommonAbstractTest { +public class IgnitePdsReserveWalSegmentsTest extends GridCommonAbstractTest { /** */ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -67,23 +67,27 @@ public class IgnitePdsUnusedWalSegmentsTest extends GridCommonAbstractTest { cfg.setDataStorageConfiguration(dbCfg); dbCfg.setWalSegmentSize(1024 * 1024) - .setWalHistorySize(Integer.MAX_VALUE) - .setWalSegments(10) - .setWalMode(WALMode.LOG_ONLY) - .setDefaultDataRegionConfiguration(new DataRegionConfiguration() - .setMaxSize(100 * 1024 * 1024) - .setPersistenceEnabled(true)); + .setMaxWalArchiveSize(Long.MAX_VALUE) + .setWalSegments(10) + .setWalMode(WALMode.LOG_ONLY) + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(100 * 1024 * 1024) + .setPersistenceEnabled(true)); return cfg; } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { + stopAllGrids(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { + stopAllGrids(); + cleanPersistenceDir(); } @@ -93,32 +97,27 @@ public class IgnitePdsUnusedWalSegmentsTest extends GridCommonAbstractTest { * @throws Exception if failed. */ public void testWalManagerRangeReservation() throws Exception { - try{ - IgniteEx ig0 = prepareGrid(4); + IgniteEx ig0 = prepareGrid(4); - GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager) ig0.context().cache().context() - .database(); + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)ig0.context().cache().context() + .database(); - IgniteWriteAheadLogManager wal = ig0.context().cache().context().wal(); + IgniteWriteAheadLogManager wal = ig0.context().cache().context().wal(); - long resIdx = getReservedWalSegmentIndex(dbMgr); + long resIdx = getReservedWalSegmentIndex(dbMgr); - assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); + assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); FileWALPointer lowPtr = (FileWALPointer)dbMgr.checkpointHistory().firstCheckpointPointer(); - assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); + assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); - // Reserve previous WAL segment. - wal.reserve(new FileWALPointer(resIdx - 1, 0, 0)); + // Reserve previous WAL segment. + wal.reserve(new FileWALPointer(resIdx - 1, 0, 0)); - int resCnt = wal.reserved(new FileWALPointer(resIdx - 1, 0, 0), new FileWALPointer(resIdx, 0, 0)); + int resCnt = wal.reserved(new FileWALPointer(resIdx - 1, 0, 0), new FileWALPointer(resIdx, 0, 0)); - assertTrue("Expected resCnt is 2, real is " + resCnt, resCnt == 2); - } - finally { - stopAllGrids(); - } + assertTrue("Expected resCnt is 2, real is " + resCnt, resCnt == 2); } /** @@ -126,35 +125,30 @@ public void testWalManagerRangeReservation() throws Exception { * * @throws Exception if failed. */ - public void testUnusedWalTruncate() throws Exception { - try{ - IgniteEx ig0 = prepareGrid(4); + public void testWalDoesNotTruncatedWhenSegmentReserved() throws Exception { + IgniteEx ig0 = prepareGrid(4); - GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager) ig0.context().cache().context() - .database(); + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)ig0.context().cache().context() + .database(); - IgniteWriteAheadLogManager wal = ig0.context().cache().context().wal(); + IgniteWriteAheadLogManager wal = ig0.context().cache().context().wal(); - long resIdx = getReservedWalSegmentIndex(dbMgr); + long resIdx = getReservedWalSegmentIndex(dbMgr); - assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); + assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); FileWALPointer lowPtr = (FileWALPointer) dbMgr.checkpointHistory().firstCheckpointPointer(); - assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); + assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); - // Reserve previous WAL segment. - wal.reserve(new FileWALPointer(resIdx - 1, 0, 0)); + // Reserve previous WAL segment. + wal.reserve(new FileWALPointer(resIdx - 1, 0, 0)); - int numDel = wal.truncate(null, lowPtr); + int numDel = wal.truncate(null, lowPtr); - int expNumDel = (int)resIdx - 1; + int expNumDel = (int)resIdx - 1; - assertTrue("Expected del segments is " + expNumDel + ", real is " + numDel, expNumDel == numDel); - } - finally { - stopAllGrids(); - } + assertTrue("Expected del segments is " + expNumDel + ", real is " + numDel, expNumDel == numDel); } /** @@ -171,20 +165,16 @@ private IgniteEx prepareGrid(int cnt) throws Exception { IgniteCache cache = ig0.cache(DEFAULT_CACHE_NAME); - for (int k = 0; k < 10_000; k++) + for (int k = 0; k < 1_000; k++) { cache.put(k, new byte[1024]); - forceCheckpoint(); - - for (int k = 0; k < 1_000; k++) - cache.put(k, new byte[1024]); - - forceCheckpoint(); + if (k % 100 == 0) + forceCheckpoint(); + } return ig0; } - /** * Get index of reserved WAL segment by checkpointer. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWALTailIsReachedDuringIterationOverArchiveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWALTailIsReachedDuringIterationOverArchiveTest.java index 5fd5a1bee8525..e3c2c6c5fab1f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWALTailIsReachedDuringIterationOverArchiveTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWALTailIsReachedDuringIterationOverArchiveTest.java @@ -38,8 +38,8 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; -import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.FileDescriptor; import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory.IteratorParametersBuilder; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java index 938465cb3ad20..f890d61b2c025 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java @@ -33,7 +33,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; -import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; 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; @@ -163,7 +163,7 @@ private void testApplyingUpdatesFromCompactedWal(boolean switchOffCompressor) th File walDir = new File(dbDir, "wal"); File archiveDir = new File(walDir, "archive"); File nodeArchiveDir = new File(archiveDir, nodeFolderName); - File walSegment = new File(nodeArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(0) + ".zip"); + File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0) + ".zip"); assertTrue(walSegment.exists()); assertTrue(walSegment.length() < WAL_SEGMENT_SIZE / 2); // Should be compressed at least in half. @@ -268,7 +268,7 @@ private void testCompressorToleratesEmptyWalSegments(WALMode walMode) throws Exc File walDir = new File(dbDir, "wal"); File archiveDir = new File(walDir, "archive"); File nodeArchiveDir = new File(archiveDir, nodeFolderName); - File walSegment = new File(nodeArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(emptyIdx)); + File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(emptyIdx)); try (RandomAccessFile raf = new RandomAccessFile(walSegment, "rw")) { raf.setLength(0); // Clear wal segment, but don't delete. @@ -358,7 +358,7 @@ public void testSeekingStartInCompactedSegment() throws Exception { File walDir = new File(dbDir, "wal"); File archiveDir = new File(walDir, "archive"); File nodeArchiveDir = new File(archiveDir, nodeFolderName); - File walSegment = new File(nodeArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(0) + ".zip"); + File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0) + ".zip"); assertTrue(walSegment.exists()); assertTrue(walSegment.length() < WAL_SEGMENT_SIZE / 2); // Should be compressed at least in half. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java new file mode 100644 index 0000000000000..1cf237c477874 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java @@ -0,0 +1,277 @@ +/* + * 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.persistence.db.wal; + +import java.io.File; +import java.util.function.Consumer; +import java.util.stream.Stream; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointHistory; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE; + +/** + * + */ +public abstract class WalDeletionArchiveAbstractTest extends GridCommonAbstractTest { + /** */ + public static final String CACHE_NAME = "SomeCache"; + + /** + * Start grid with override default configuration via customConfigurator. + */ + private Ignite startGrid(Consumer customConfigurator) throws Exception { + IgniteConfiguration configuration = getConfiguration(getTestIgniteInstanceName()); + + DataStorageConfiguration dbCfg = new DataStorageConfiguration(); + + dbCfg.setWalMode(walMode()); + dbCfg.setWalSegmentSize(512 * 1024); + dbCfg.setCheckpointFrequency(60 * 1000);//too high value for turn off frequency checkpoint. + dbCfg.setPageSize(4 * 1024); + dbCfg.setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(100 * 1024 * 1024) + .setPersistenceEnabled(true)); + + customConfigurator.accept(dbCfg); + + configuration.setDataStorageConfiguration(dbCfg); + + Ignite ignite = startGrid(configuration); + + ignite.active(true); + + return ignite; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + System.clearProperty(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * @return WAL mode used in test. + */ + abstract protected WALMode walMode(); + + /** + * History size parameters consistency check. Should be set just one of wal history size or max wal archive size. + */ + public void testGridDoesNotStart_BecauseBothWalHistorySizeAndMaxWalArchiveSizeUsed() throws Exception { + //given: wal history size and max wal archive size are both set. + IgniteConfiguration configuration = getConfiguration(getTestIgniteInstanceName()); + + DataStorageConfiguration dbCfg = new DataStorageConfiguration(); + dbCfg.setWalHistorySize(12); + dbCfg.setMaxWalArchiveSize(9); + configuration.setDataStorageConfiguration(dbCfg); + + try { + //when: start grid. + startGrid(getTestIgniteInstanceName(), configuration); + fail("Should be fail because both wal history size and max wal archive size was used"); + } + catch (IgniteException e) { + //then: exception is occurrence because should be set just one parameters. + assertTrue(findSourceMessage(e).startsWith("Should be used only one of wal history size or max wal archive size")); + } + } + + /** + * find first cause's message + */ + private String findSourceMessage(Throwable ex) { + return ex.getCause() == null ? ex.getMessage() : findSourceMessage(ex.getCause()); + } + + /** + * Correct delete archived wal files. + */ + public void testCorrectDeletedArchivedWalFiles() throws Exception { + //given: configured grid with setted max wal archive size + long maxWalArchiveSize = 2 * 1024 * 1024; + Ignite ignite = startGrid(dbCfg -> { + dbCfg.setMaxWalArchiveSize(maxWalArchiveSize); + }); + + GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); + + long allowedThresholdWalArchiveSize = maxWalArchiveSize / 2; + + IgniteCache cache = ignite.getOrCreateCache(CACHE_NAME); + + //when: put to cache more than 2 MB + for (int i = 0; i < 500; i++) + cache.put(i, i); + + forceCheckpoint(); + + //then: total archive size less than half of maxWalArchiveSize(by current logic) + IgniteWriteAheadLogManager wal = wal(ignite); + + FileDescriptor[] files = (FileDescriptor[])U.findNonPublicMethod(wal.getClass(), "walArchiveFiles").invoke(wal); + + Long totalSize = Stream.of(files) + .map(desc -> desc.file().length()) + .reduce(0L, Long::sum); + + assertTrue(files.length >= 1); + assertTrue(totalSize <= allowedThresholdWalArchiveSize); + assertFalse(Stream.of(files).anyMatch(desc -> desc.file().getName().endsWith("00001.wal"))); + + CheckpointHistory hist = dbMgr.checkpointHistory(); + + assertTrue(hist.checkpoints().size() > 0); + } + + /** + * Checkpoint triggered depends on wal size. + */ + public void testCheckpointStarted_WhenWalHasTooBigSizeWithoutCheckpoint() throws Exception { + //given: configured grid with max wal archive size = 1MB, wal segment size = 512KB + Ignite ignite = startGrid(dbCfg -> { + dbCfg.setMaxWalArchiveSize(1 * 1024 * 1024);// 1 Mbytes + }); + + GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); + + IgniteCache cache = ignite.getOrCreateCache(CACHE_NAME); + + for (int i = 0; i < 500; i++) + cache.put(i, i); + + //then: checkpoint triggered by size limit of wall without checkpoint + GridCacheDatabaseSharedManager.Checkpointer checkpointer = dbMgr.getCheckpointer(); + + String checkpointReason = U.field((Object)U.field(checkpointer, "curCpProgress"), "reason"); + + assertEquals("too big size of WAL without checkpoint", checkpointReason); + } + + /** + * Test for check deprecated removing checkpoint by deprecated walHistorySize parameter + * + * @deprecated Test old removing process depends on WalHistorySize. + */ + public void testCheckpointHistoryRemovingByWalHistorySize() throws Exception { + //given: configured grid with wal history size = 10 + int walHistorySize = 10; + + Ignite ignite = startGrid(dbCfg -> { + dbCfg.setWalHistorySize(walHistorySize); + }); + + GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); + + IgniteCache cache = ignite.getOrCreateCache(CACHE_NAME); + + //when: put to cache and do checkpoint + int testNumberOfCheckpoint = walHistorySize * 2; + + for (int i = 0; i < testNumberOfCheckpoint; i++) { + cache.put(i, i); + //and: wait for checkpoint finished + forceCheckpoint(); + } + + //then: number of checkpoints less or equal than walHistorySize + CheckpointHistory hist = dbMgr.checkpointHistory(); + assertTrue(hist.checkpoints().size() == walHistorySize); + + File[] cpFiles = dbMgr.checkpointDirectory().listFiles(); + + assertTrue(cpFiles.length <= (walHistorySize * 2 + 1));// starts & ends + node_start + } + + /** + * Correct delete checkpoint history from memory depends on IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE. WAL files + * doesn't delete because deleting was disabled. + */ + public void testCorrectDeletedCheckpointHistoryButKeepWalFiles() throws Exception { + System.setProperty(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, "2"); + //given: configured grid with disabled WAL removing. + Ignite ignite = startGrid(dbCfg -> { + dbCfg.setMaxWalArchiveSize(Long.MAX_VALUE); + }); + + GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); + + IgniteCache cache = ignite.getOrCreateCache(CACHE_NAME); + + //when: put to cache + for (int i = 0; i < 500; i++) { + cache.put(i, i); + + if (i % 10 == 0) + forceCheckpoint(); + } + + forceCheckpoint(); + + //then: WAL files was not deleted but some of checkpoint history was deleted. + IgniteWriteAheadLogManager wal = wal(ignite); + + FileDescriptor[] files = (FileDescriptor[])U.findNonPublicMethod(wal.getClass(), "walArchiveFiles").invoke(wal); + + boolean hasFirstSegment = Stream.of(files) + .anyMatch(desc -> desc.file().getName().endsWith("0001.wal")); + + assertTrue(hasFirstSegment); + + CheckpointHistory hist = dbMgr.checkpointHistory(); + + assertTrue(hist.checkpoints().size() == 2); + } + + /** + * Extract GridCacheDatabaseSharedManager. + */ + private GridCacheDatabaseSharedManager gridDatabase(Ignite ignite) { + return (GridCacheDatabaseSharedManager)((IgniteEx)ignite).context().cache().context().database(); + } + + /** + * Extract IgniteWriteAheadLogManager. + */ + private IgniteWriteAheadLogManager wal(Ignite ignite) { + return ((IgniteEx)ignite).context().cache().context().wal(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveFsyncTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveFsyncTest.java new file mode 100644 index 0000000000000..64a31854cf86f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveFsyncTest.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.processors.cache.persistence.db.wal; + +import org.apache.ignite.configuration.WALMode; + +/** + * + */ +public class WalDeletionArchiveFsyncTest extends WalDeletionArchiveAbstractTest { + + /** {@inheritDoc} */ + @Override protected WALMode walMode() { + return WALMode.FSYNC; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveLogOnlyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveLogOnlyTest.java new file mode 100644 index 0000000000000..b1cdf7a42796d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveLogOnlyTest.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.processors.cache.persistence.db.wal; + +import org.apache.ignite.configuration.WALMode; + +/** + * + */ +public class WalDeletionArchiveLogOnlyTest extends WalDeletionArchiveAbstractTest { + + /** {@inheritDoc} */ + @Override protected WALMode walMode() { + return WALMode.LOG_ONLY; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java index 5abcff4cca636..f84e2b9d8279b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java @@ -472,50 +472,6 @@ public void testHistoricalRebalanceIterator() throws Exception { } } - /** - * @throws Exception if failed. - */ - public void testCheckpointHistory() throws Exception { - Ignite ignite = startGrid(); - - ignite.cluster().active(true); - - try { - GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)((IgniteEx)ignite).context() - .cache().context().database(); - - dbMgr.waitForCheckpoint("test"); - - // This number depends on wal history size. - int entries = WAL_HIST_SIZE * 2; - - IgniteCache cache = ignite.cache(CACHE_NAME); - - for (int i = 0; i < entries; i++) { - // Put to partition 0. - cache.put(i * PARTS, i * PARTS); - - // Put to partition 1. - cache.put(i * PARTS + 1, i * PARTS + 1); - - dbMgr.waitForCheckpoint("test"); - } - - CheckpointHistory hist = dbMgr.checkpointHistory(); - - assertTrue(hist.checkpoints().size() <= WAL_HIST_SIZE); - - File cpDir = dbMgr.checkpointDirectory(); - - File[] cpFiles = cpDir.listFiles(); - - assertTrue(cpFiles.length <= WAL_HIST_SIZE * 2 + 1); // starts & ends + node_start - } - finally { - stopAllGrids(); - } - } - /** * @throws Exception If failed. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java index df649fa257a44..b78337315f801 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java @@ -94,6 +94,7 @@ public WALIterator iterator(File wal, File walArchive) throws IgniteCheckedExcep when(persistentCfg1.getWalSegments()).thenReturn(segments); when(persistentCfg1.getWalBufferSize()).thenReturn(DataStorageConfiguration.DFLT_WAL_BUFF_SIZE); when(persistentCfg1.getWalRecordIteratorBufferSize()).thenReturn(DataStorageConfiguration.DFLT_WAL_RECORD_ITERATOR_BUFFER_SIZE); + when(persistentCfg1.getWalSegmentSize()).thenReturn(DataStorageConfiguration.DFLT_WAL_SEGMENT_SIZE); final FileIOFactory fileIOFactory = new DataStorageConfiguration().getFileIOFactory(); when(persistentCfg1.getFileIOFactory()).thenReturn(fileIOFactory); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java index 3d62fe1c0e542..811a231524522 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; import org.apache.ignite.lang.IgniteFuture; /** @@ -160,4 +161,9 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { @Override public long lastCompactedSegment() { return -1L; } + + /** {@inheritDoc} */ + @Override public long maxArchivedSegmentToDelete() { + return -1; + } } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java index 2614f1d0cd8f8..5eba5a4511fe2 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java @@ -27,23 +27,24 @@ import org.apache.ignite.internal.processors.cache.persistence.IgniteRebalanceScheduleResendPartitionsTest; import org.apache.ignite.internal.processors.cache.persistence.LocalWacModeNoChangeDuringRebalanceOnNonNodeAssignTest; import org.apache.ignite.internal.processors.cache.persistence.LocalWalModeChangeDuringRebalancingSelfTest; -import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteAbsentEvictionNodeOutOfBaselineTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.ClientAffinityAssignmentWithBaselineTest; +import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteAbsentEvictionNodeOutOfBaselineTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteAllBaselineNodesOnlineFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteOfflineBaselineNodeFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteOnlineNodeOutOfBaselineFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsRebalancingOnNotStableTopologyTest; +import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsReserveWalSegmentsTest; +import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTransactionsHangTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWholeClusterRestartTest; import org.apache.ignite.internal.processors.cache.persistence.db.SlowHistoricalRebalanceSmallHistoryTest; import org.apache.ignite.internal.processors.cache.persistence.db.checkpoint.IgniteCheckpointDirtyPagesForLowLoadTest; -import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsUnusedWalSegmentsTest; import org.apache.ignite.internal.processors.cache.persistence.db.filename.IgniteUidAsConsistentIdMigrationTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteNodeStoppedDuringDisableWALTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWALTailIsReachedDuringIterationOverArchiveTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushBackgroundSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushBackgroundWithMmapBufferSelfTest; -import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFailoverTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncWithDedicatedWorkerSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncWithMmapBufferSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlySelfTest; @@ -55,6 +56,8 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoverySeveralRestartsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalCompactionTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalDeletionArchiveFsyncTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalDeletionArchiveLogOnlyTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteDataIntegrityTests; import org.apache.ignite.internal.processors.cache.persistence.db.wal.reader.IgniteWalReaderTest; @@ -146,7 +149,7 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgnitePdsExchangeDuringCheckpointTest.class); - suite.addTestSuite(IgnitePdsUnusedWalSegmentsTest.class); + suite.addTestSuite(IgnitePdsReserveWalSegmentsTest.class); // new style folders with generated consistent ID test suite.addTestSuite(IgniteUidAsConsistentIdMigrationTest.class); @@ -155,6 +158,9 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(WalCompactionTest.class); + suite.addTestSuite(WalDeletionArchiveFsyncTest.class); + suite.addTestSuite(WalDeletionArchiveLogOnlyTest.class); + suite.addTestSuite(IgniteCheckpointDirtyPagesForLowLoadTest.class); suite.addTestSuite(IgnitePdsCorruptedStoreTest.class); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageConfigurationParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageConfigurationParityTest.cs index 51f2865ad5bda..2276c2027cced 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageConfigurationParityTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageConfigurationParityTest.cs @@ -28,7 +28,8 @@ public class DataStorageConfigurationParityTest /** Properties that are not needed on .NET side. */ private static readonly string[] UnneededProperties = { - "FileIOFactory" + "FileIOFactory", + "isWalHistorySizeParameterUsed" }; /** Properties that are missing on .NET side. */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs index 7a3778c7ea426..0a010b4a75c6d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs @@ -163,6 +163,11 @@ public class DataStorageConfiguration /// public const int DefaultConcurrencyLevel = 0; + /// + /// Default value for . + /// + public const long DefaultMaxWalArchiveSize = 1024 * 1024 * 1024; + /// /// Initializes a new instance of the class. /// @@ -190,6 +195,7 @@ public DataStorageConfiguration() SystemRegionMaxSize = DefaultSystemRegionMaxSize; PageSize = DefaultPageSize; WalAutoArchiveAfterInactivity = DefaultWalAutoArchiveAfterInactivity; + MaxWalArchiveSize = DefaultMaxWalArchiveSize; } /// @@ -221,6 +227,7 @@ internal DataStorageConfiguration(IBinaryRawReader reader) CheckpointWriteOrder = (CheckpointWriteOrder)reader.ReadInt(); WriteThrottlingEnabled = reader.ReadBoolean(); WalCompactionEnabled = reader.ReadBoolean(); + MaxWalArchiveSize = reader.ReadLong(); SystemRegionInitialSize = reader.ReadLong(); SystemRegionMaxSize = reader.ReadLong(); @@ -272,6 +279,7 @@ internal void Write(IBinaryRawWriter writer) writer.WriteInt((int)CheckpointWriteOrder); writer.WriteBoolean(WriteThrottlingEnabled); writer.WriteBoolean(WalCompactionEnabled); + writer.WriteLong(MaxWalArchiveSize); writer.WriteLong(SystemRegionInitialSize); writer.WriteLong(SystemRegionMaxSize); @@ -444,6 +452,12 @@ internal void Write(IBinaryRawWriter writer) [DefaultValue(DefaultWalCompactionEnabled)] public bool WalCompactionEnabled { get; set; } + /// + /// Gets or sets maximum size of wal archive folder, in bytes. + /// + [DefaultValue(DefaultMaxWalArchiveSize)] + public long MaxWalArchiveSize { get; set; } + /// /// Gets or sets the size of a memory chunk reserved for system needs. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index 8db5afa2f3154..ebbef67334ca9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -1855,6 +1855,11 @@ If true, system will filter and compress WAL archive in background. Compressed WAL archive gets automatically decompressed on demand. + + + Maximum size of wal archive folder, in bytes. + + Size of the memory page. From 8af84fe0f62880937b7350efa9a9646086a98ce3 Mon Sep 17 00:00:00 2001 From: EdShangGG Date: Thu, 30 Aug 2018 16:47:23 +0300 Subject: [PATCH 21/95] IGNITE-9302 Added timeouts for Java Thin Clients tests - Fixes #4562. Signed-off-by: Alexey Goncharuk --- .../ignite/client/ClientCacheConfigurationTest.java | 7 +++++++ .../ignite/client/ClientConfigurationTest.java | 7 +++++++ .../org/apache/ignite/client/FunctionalTest.java | 7 +++++++ .../org/apache/ignite/client/IgniteBinaryTest.java | 11 +++++++---- .../test/java/org/apache/ignite/client/LoadTest.java | 10 +++++++--- .../org/apache/ignite/client/ReliabilityTest.java | 12 +++++++----- .../cluster/FullyConnectedComponentSearcherTest.java | 7 +++++++ .../persistence/pagemem/FullPageIdTableTest.java | 7 +++++++ .../IgnitePageMemReplaceDelayedWriteUnitTest.java | 7 +++++++ .../pagemem/IgniteThrottlingUnitTest.java | 7 +++++++ .../pagemem/RobinHoodBackwardShiftHashMapTest.java | 6 ++++++ .../apache/ignite/testframework/GridTestUtils.java | 3 +++ .../testframework/junits/GridAbstractTest.java | 5 +---- .../apache/ignite/client/FunctionalQueryTest.java | 7 +++++++ .../apache/ignite/client/IgniteBinaryQueryTest.java | 7 +++++++ .../java/org/apache/ignite/client/SecurityTest.java | 7 +++++++ .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java | 6 ++++++ 17 files changed, 107 insertions(+), 16 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/client/ClientCacheConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/client/ClientCacheConfigurationTest.java index 0c82b26f7cea3..350c0dc3f9b24 100644 --- a/modules/core/src/test/java/org/apache/ignite/client/ClientCacheConfigurationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/client/ClientCacheConfigurationTest.java @@ -38,7 +38,10 @@ import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.configuration.ClientConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertTrue; @@ -46,6 +49,10 @@ * {@link ClientConfiguration} unit tests. */ public class ClientCacheConfigurationTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** Serialization/deserialization. */ @Test public void testSerialization() throws IOException, ClassNotFoundException { diff --git a/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java index e6ab4d73b1c08..bcc212abbe130 100644 --- a/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java @@ -27,7 +27,10 @@ import java.util.Collections; import org.apache.ignite.configuration.BinaryConfiguration; import org.apache.ignite.configuration.ClientConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertTrue; @@ -35,6 +38,10 @@ * {@link ClientConfiguration} unit tests. */ public class ClientConfigurationTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** Serialization/deserialization. */ @Test public void testSerialization() throws IOException, ClassNotFoundException { diff --git a/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java b/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java index b49f7e3a186f8..1272287d3189d 100644 --- a/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java +++ b/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java @@ -40,7 +40,10 @@ import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.configuration.ClientConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -54,6 +57,10 @@ * Thin client functional tests. */ public class FunctionalTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** * Tested API: *
      diff --git a/modules/core/src/test/java/org/apache/ignite/client/IgniteBinaryTest.java b/modules/core/src/test/java/org/apache/ignite/client/IgniteBinaryTest.java index b9402ccb65807..0f84c75dcd426 100644 --- a/modules/core/src/test/java/org/apache/ignite/client/IgniteBinaryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/client/IgniteBinaryTest.java @@ -26,12 +26,11 @@ import org.apache.ignite.Ignition; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.client.ClientCache; -import org.apache.ignite.client.Config; -import org.apache.ignite.client.IgniteClient; -import org.apache.ignite.client.Person; import org.apache.ignite.configuration.ClientConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -40,6 +39,10 @@ * Ignite {@link BinaryObject} API system tests. */ public class IgniteBinaryTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** * Unmarshalling schema-less Ignite binary objects into Java static types. */ diff --git a/modules/core/src/test/java/org/apache/ignite/client/LoadTest.java b/modules/core/src/test/java/org/apache/ignite/client/LoadTest.java index 63bcf57f7c352..f97e7b782e7f2 100644 --- a/modules/core/src/test/java/org/apache/ignite/client/LoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/client/LoadTest.java @@ -39,10 +39,10 @@ import org.apache.ignite.configuration.BinaryConfiguration; import org.apache.ignite.configuration.ClientConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.client.ClientCache; -import org.apache.ignite.client.Config; -import org.apache.ignite.client.IgniteClient; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -52,6 +52,10 @@ * Load, capacity and performance tests. */ public class LoadTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** * Test thin client in multi-thread environment. */ diff --git a/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java b/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java index 147f371ab70c1..f019fd9ca2475 100644 --- a/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java +++ b/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java @@ -34,14 +34,12 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.internal.processors.platform.client.ClientStatus; -import org.apache.ignite.client.ClientCache; -import org.apache.ignite.client.ClientCacheConfiguration; -import org.apache.ignite.client.IgniteClient; import org.apache.ignite.configuration.ClientConfiguration; import org.apache.ignite.internal.client.thin.ClientServerError; -import org.apache.ignite.client.ClientConnectionException; -import org.apache.ignite.client.LocalIgniteCluster; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -51,6 +49,10 @@ * High Availability tests. */ public class ReliabilityTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** * Thin clint failover. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/cluster/FullyConnectedComponentSearcherTest.java b/modules/core/src/test/java/org/apache/ignite/internal/cluster/FullyConnectedComponentSearcherTest.java index d6680cf0867f8..7042da0c10a39 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/cluster/FullyConnectedComponentSearcherTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/cluster/FullyConnectedComponentSearcherTest.java @@ -23,9 +23,12 @@ import java.util.Random; import org.apache.ignite.internal.cluster.graph.FullyConnectedComponentSearcher; import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.NotNull; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -34,6 +37,10 @@ */ @RunWith(Parameterized.class) public class FullyConnectedComponentSearcherTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** Adjacency matrix provider for each test. */ private AdjacencyMatrixProvider provider; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTableTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTableTest.java index 43b27aa439f2c..e337bb13ab50f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTableTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTableTest.java @@ -26,7 +26,10 @@ import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.logger.java.JavaLogger; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertEquals; @@ -34,6 +37,10 @@ * */ public class FullPageIdTableTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** */ private static final int CACHE_ID_RANGE = 1; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java index c6f42e16e87f2..aa1e37d594e86 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java @@ -42,8 +42,11 @@ import org.apache.ignite.internal.util.GridMultiCollectionWrapper; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.logger.NullLogger; +import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.NotNull; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import org.mockito.Mockito; import static org.mockito.Matchers.any; @@ -54,6 +57,10 @@ * Unit test for delayed page replacement mode. */ public class IgnitePageMemReplaceDelayedWriteUnitTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** CPU count. */ private static final int CPUS = 32; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java index f9ca7e43ad3fb..6f504da503461 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java @@ -24,7 +24,10 @@ import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker; import org.apache.ignite.internal.processors.cache.persistence.CheckpointWriteProgressSupplier; import org.apache.ignite.logger.NullLogger; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -37,6 +40,10 @@ * */ public class IgniteThrottlingUnitTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** Logger. */ private IgniteLogger log = new NullLogger(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/RobinHoodBackwardShiftHashMapTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/RobinHoodBackwardShiftHashMapTest.java index 565b99ee0161a..345651e59469e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/RobinHoodBackwardShiftHashMapTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/RobinHoodBackwardShiftHashMapTest.java @@ -26,8 +26,11 @@ import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.NotNull; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.apache.ignite.IgniteSystemProperties.IGNITE_LONG_LONG_HASH_MAP_LOAD_FACTOR; import static org.junit.Assert.assertEquals; @@ -38,6 +41,9 @@ * Unit tests of {@link RobinHoodBackwardShiftHashMap} implementation. */ public class RobinHoodBackwardShiftHashMapTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); /** * @param tester map test code diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java index cb3d8f285aa42..c6d9527fa1cb8 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java @@ -114,6 +114,9 @@ public final class GridTestUtils { /** Default busy wait sleep interval in milliseconds. */ public static final long DFLT_BUSYWAIT_SLEEP_INTERVAL = 200; + /** */ + public static final long DFLT_TEST_TIMEOUT = 5 * 60 * 1000; + /** */ static final String ALPHABETH = "qwertyuiopasdfghjklzxcvbnmQWERTYUIOPASDFGHJKLZXCVBNM1234567890_"; 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 09475e6808692..a1d6c2528771a 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 @@ -159,9 +159,6 @@ public abstract class GridAbstractTest extends TestCase { setAddresses(Collections.singleton("127.0.0.1:47500..47509")); }}; - /** */ - private static final long DFLT_TEST_TIMEOUT = 5 * 60 * 1000; - /** */ private static final int DFLT_TOP_WAIT_TIMEOUT = 2000; @@ -2183,7 +2180,7 @@ private long getDefaultTestTimeout() { if (timeout != null) return Long.parseLong(timeout); - return DFLT_TEST_TIMEOUT; + return GridTestUtils.DFLT_TEST_TIMEOUT; } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/client/FunctionalQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/client/FunctionalQueryTest.java index eeec30f9e5d9b..d10ed1a71297d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/client/FunctionalQueryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/client/FunctionalQueryTest.java @@ -36,7 +36,10 @@ import org.apache.ignite.configuration.BinaryConfiguration; import org.apache.ignite.configuration.ClientConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.*; @@ -44,6 +47,10 @@ * Thin client functional tests. */ public class FunctionalQueryTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** * Tested API: *
        diff --git a/modules/indexing/src/test/java/org/apache/ignite/client/IgniteBinaryQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/client/IgniteBinaryQueryTest.java index 4b3eebcc0e0d8..a524394860d1d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/client/IgniteBinaryQueryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/client/IgniteBinaryQueryTest.java @@ -38,7 +38,10 @@ import org.apache.ignite.client.Config; import org.apache.ignite.client.IgniteClient; import org.apache.ignite.configuration.ClientConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertEquals; @@ -46,6 +49,10 @@ * Ignite {@link BinaryObject} API system tests. */ public class IgniteBinaryQueryTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** * Test queries in Ignite binary. */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java b/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java index e2b11db2f66e0..0be13db0ca8ea 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java @@ -31,8 +31,11 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.ssl.SslContextFactory; +import org.apache.ignite.testframework.GridTestUtils; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -41,6 +44,10 @@ * Thin client security test. */ public class SecurityTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** Ignite home. */ private static final String IGNITE_HOME = U.getIgniteHome(); diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java index 25598fef8fa93..f85c6a3c9b7d1 100644 --- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java @@ -36,6 +36,8 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Rule; +import org.junit.rules.Timeout; /** * Test for {@link TcpDiscoveryZookeeperIpFinder}. @@ -43,6 +45,10 @@ * @author Raul Kripalani */ public class ZookeeperIpFinderTest extends GridCommonAbstractTest { + /** Per test timeout */ + @Rule + public Timeout globalTimeout = new Timeout((int) GridTestUtils.DFLT_TEST_TIMEOUT); + /** ZK Cluster size. */ private static final int ZK_CLUSTER_SIZE = 3; From 1d4ccec375a67fa0abd3b6beaa5d80ded8ed8424 Mon Sep 17 00:00:00 2001 From: Alexey Platonov Date: Thu, 30 Aug 2018 19:28:34 +0300 Subject: [PATCH 22/95] IGNITE-9237: [ML] Random forest optimization this closes #4612 --- .../RandomForestClassificationExample.java | 386 +++++++++--------- .../RandomForestRegressionExample.java | 26 +- modules/ml/pom.xml | 13 +- .../WeightedPredictionsAggregator.java | 2 +- .../ignite/ml/dataset/feature/BucketMeta.java | 88 ++++ .../dataset/feature/DistributionComputer.java | 32 ++ .../ml/dataset/feature/FeatureMeta.java | 65 +++ .../ignite/ml/dataset/feature/Histogram.java | 64 +++ .../ml/dataset/feature/ObjectHistogram.java | 129 ++++++ .../ml/dataset/feature/package-info.java | 24 ++ .../BootstrappedDatasetBuilder.java | 90 ++++ .../BootstrappedDatasetPartition.java | 68 +++ .../bootstrapping/BootstrappedVector.java | 87 ++++ .../impl/bootstrapping/package-info.java | 23 ++ .../RandomForestClassifierTrainer.java | 117 +++--- .../RandomForestRegressionTrainer.java | 82 ++-- .../randomforest/RandomForestTrainer.java | 372 +++++++++++++++-- .../FeaturesCountSelectionStrategies.java | 47 +++ .../ml/tree/randomforest/data/NodeId.java | 55 +++ .../ml/tree/randomforest/data/NodeSplit.java | 79 ++++ .../ml/tree/randomforest/data/TreeNode.java | 200 +++++++++ .../ml/tree/randomforest/data/TreeRoot.java | 86 ++++ .../data/impurity/GiniHistogram.java | 225 ++++++++++ .../data/impurity/GiniHistogramsComputer.java | 46 +++ .../data/impurity/ImpurityComputer.java | 38 ++ .../data/impurity/ImpurityHistogram.java | 74 ++++ .../impurity/ImpurityHistogramsComputer.java | 211 ++++++++++ .../data/impurity/MSEHistogram.java | 235 +++++++++++ .../data/impurity/MSEHistogramComputer.java | 33 ++ .../data/impurity/package-info.java | 22 + .../tree/randomforest/data/package-info.java | 22 + .../ClassifierLeafValuesComputer.java | 82 ++++ .../data/statistics/LeafValuesComputer.java | 158 +++++++ .../data/statistics/MeanValueStatistic.java | 72 ++++ .../NormalDistributionStatistics.java | 112 +++++ .../NormalDistributionStatisticsComputer.java | 119 ++++++ .../RegressionLeafValuesComputer.java | 62 +++ .../data/statistics/package-info.java | 22 + .../ignite/ml/dataset/DatasetTestSuite.java | 4 +- .../dataset/feature/ObjectHistogramTest.java | 170 ++++++++ .../ml/environment/EnvironmentTestSuite.java | 1 - .../environment/LearningEnvironmentTest.java | 187 --------- .../ignite/ml/tree/DecisionTreeTestSuite.java | 6 +- .../RandomForestClassifierTrainerTest.java | 26 +- .../RandomForestRegressionTrainerTest.java | 17 +- .../tree/randomforest/RandomForestTest.java | 78 ++++ .../RandomForestTreeTestSuite.java | 39 ++ .../tree/randomforest/data/TreeNodeTest.java | 79 ++++ .../impurity/GiniFeatureHistogramTest.java | 254 ++++++++++++ .../data/impurity/ImpurityHistogramTest.java | 69 ++++ .../data/impurity/MSEHistogramTest.java | 135 ++++++ ...malDistributionStatisticsComputerTest.java | 130 ++++++ 52 files changed, 4309 insertions(+), 554 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/BucketMeta.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/DistributionComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/FeatureMeta.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/Histogram.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/ObjectHistogram.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetPartition.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedVector.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/FeaturesCountSelectionStrategies.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeId.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeSplit.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeNode.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeRoot.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogram.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogramsComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogram.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogram.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/ClassifierLeafValuesComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/LeafValuesComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/MeanValueStatistic.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatistics.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/RegressionLeafValuesComputer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/package-info.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/dataset/feature/ObjectHistogramTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTreeTestSuite.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/TreeNodeTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniFeatureHistogramTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputerTest.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java index 09c1e4f1a48d4..4ea471cbf8c63 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java @@ -18,6 +18,9 @@ package org.apache.ignite.examples.ml.tree.randomforest; import java.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -26,18 +29,20 @@ import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.examples.ml.util.TestCache; import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.tree.randomforest.RandomForestClassifierTrainer; import org.apache.ignite.ml.tree.randomforest.RandomForestTrainer; +import org.apache.ignite.ml.tree.randomforest.data.FeaturesCountSelectionStrategies; import org.apache.ignite.thread.IgniteThread; /** - * Example represents a solution for the task of wine classification based on RandomForest - * implementation for multi-classification. It shows an initialization of {@link RandomForestTrainer} with - * thread pool for multi-thread learning, initialization of Ignite Cache, - * learning step and evaluation of accuracy of model. + * Example represents a solution for the task of wine classification based on RandomForestTrainer implementation for + * multi-classification. It shows an initialization of {@link RandomForestTrainer} with thread pool for multi-thread + * learning, initialization of Ignite Cache, learning step and evaluation of accuracy of model. * * Dataset url: https://archive.ics.uci.edu/ml/machine-learning-databases/wine/ + * * @see RandomForestClassifierTrainer */ public class RandomForestClassificationExample { @@ -55,11 +60,18 @@ public static void main(String[] args) throws InterruptedException { RandomForestClassificationExample.class.getSimpleName(), () -> { IgniteCache dataCache = new TestCache(ignite).fillCacheWith(data); - RandomForestClassifierTrainer trainer = new RandomForestClassifierTrainer(13, 4, 101, 0.3, 2, 0); + AtomicInteger indx = new AtomicInteger(0); + RandomForestClassifierTrainer classifier = new RandomForestClassifierTrainer( + IntStream.range(0, 13).mapToObj(x -> new FeatureMeta("", indx.getAndIncrement(), false)).collect(Collectors.toList())) + .withCountOfTrees(101) + .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.SQRT) + .withMaxDepth(3) + .withMinImpurityDelta(0.) + .withSubsampleSize(0.3); - ModelsComposition randomForest = trainer.fit(ignite, dataCache, - (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), - (k, v) -> v[0] + ModelsComposition randomForest = classifier.fit(ignite, dataCache, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] ); int amountOfErrors = 0; @@ -80,7 +92,7 @@ public static void main(String[] args) throws InterruptedException { } System.out.println("\n>>> Absolute amount of errors " + amountOfErrors); - System.out.println("\n>>> Accuracy " + (1 - amountOfErrors / (double) totalAmount)); + System.out.println("\n>>> Accuracy " + (1 - amountOfErrors / (double)totalAmount)); } }); @@ -93,183 +105,183 @@ public static void main(String[] args) throws InterruptedException { * The Wine dataset. */ private static final double[][] data = { - {1, 14.23, 1.71, 2.43, 15.6, 127, 2.8, 3.06, .28, 2.29, 5.64, 1.04, 3.92, 1065}, - {1, 13.2, 1.78, 2.14, 11.2, 100, 2.65, 2.76, .26, 1.28, 4.38, 1.05, 3.4, 1050}, - {1, 13.16, 2.36, 2.67, 18.6, 101, 2.8, 3.24, .3, 2.81, 5.68, 1.03, 3.17, 1185}, - {1, 14.37, 1.95, 2.5, 16.8, 113, 3.85, 3.49, .24, 2.18, 7.8, .86, 3.45, 1480}, - {1, 13.24, 2.59, 2.87, 21, 118, 2.8, 2.69, .39, 1.82, 4.32, 1.04, 2.93, 735}, - {1, 14.2, 1.76, 2.45, 15.2, 112, 3.27, 3.39, .34, 1.97, 6.75, 1.05, 2.85, 1450}, - {1, 14.39, 1.87, 2.45, 14.6, 96, 2.5, 2.52, .3, 1.98, 5.25, 1.02, 3.58, 1290}, - {1, 14.06, 2.15, 2.61, 17.6, 121, 2.6, 2.51, .31, 1.25, 5.05, 1.06, 3.58, 1295}, - {1, 14.83, 1.64, 2.17, 14, 97, 2.8, 2.98, .29, 1.98, 5.2, 1.08, 2.85, 1045}, - {1, 13.86, 1.35, 2.27, 16, 98, 2.98, 3.15, .22, 1.85, 7.22, 1.01, 3.55, 1045}, - {1, 14.1, 2.16, 2.3, 18, 105, 2.95, 3.32, .22, 2.38, 5.75, 1.25, 3.17, 1510}, - {1, 14.12, 1.48, 2.32, 16.8, 95, 2.2, 2.43, .26, 1.57, 5, 1.17, 2.82, 1280}, - {1, 13.75, 1.73, 2.41, 16, 89, 2.6, 2.76, .29, 1.81, 5.6, 1.15, 2.9, 1320}, - {1, 14.75, 1.73, 2.39, 11.4, 91, 3.1, 3.69, .43, 2.81, 5.4, 1.25, 2.73, 1150}, - {1, 14.38, 1.87, 2.38, 12, 102, 3.3, 3.64, .29, 2.96, 7.5, 1.2, 3, 1547}, - {1, 13.63, 1.81, 2.7, 17.2, 112, 2.85, 2.91, .3, 1.46, 7.3, 1.28, 2.88, 1310}, - {1, 14.3, 1.92, 2.72, 20, 120, 2.8, 3.14, .33, 1.97, 6.2, 1.07, 2.65, 1280}, - {1, 13.83, 1.57, 2.62, 20, 115, 2.95, 3.4, .4, 1.72, 6.6, 1.13, 2.57, 1130}, - {1, 14.19, 1.59, 2.48, 16.5, 108, 3.3, 3.93, .32, 1.86, 8.7, 1.23, 2.82, 1680}, - {1, 13.64, 3.1, 2.56, 15.2, 116, 2.7, 3.03, .17, 1.66, 5.1, .96, 3.36, 845}, - {1, 14.06, 1.63, 2.28, 16, 126, 3, 3.17, .24, 2.1, 5.65, 1.09, 3.71, 780}, - {1, 12.93, 3.8, 2.65, 18.6, 102, 2.41, 2.41, .25, 1.98, 4.5, 1.03, 3.52, 770}, - {1, 13.71, 1.86, 2.36, 16.6, 101, 2.61, 2.88, .27, 1.69, 3.8, 1.11, 4, 1035}, - {1, 12.85, 1.6, 2.52, 17.8, 95, 2.48, 2.37, .26, 1.46, 3.93, 1.09, 3.63, 1015}, - {1, 13.5, 1.81, 2.61, 20, 96, 2.53, 2.61, .28, 1.66, 3.52, 1.12, 3.82, 845}, - {1, 13.05, 2.05, 3.22, 25, 124, 2.63, 2.68, .47, 1.92, 3.58, 1.13, 3.2, 830}, - {1, 13.39, 1.77, 2.62, 16.1, 93, 2.85, 2.94, .34, 1.45, 4.8, .92, 3.22, 1195}, - {1, 13.3, 1.72, 2.14, 17, 94, 2.4, 2.19, .27, 1.35, 3.95, 1.02, 2.77, 1285}, - {1, 13.87, 1.9, 2.8, 19.4, 107, 2.95, 2.97, .37, 1.76, 4.5, 1.25, 3.4, 915}, - {1, 14.02, 1.68, 2.21, 16, 96, 2.65, 2.33, .26, 1.98, 4.7, 1.04, 3.59, 1035}, - {1, 13.73, 1.5, 2.7, 22.5, 101, 3, 3.25, .29, 2.38, 5.7, 1.19, 2.71, 1285}, - {1, 13.58, 1.66, 2.36, 19.1, 106, 2.86, 3.19, .22, 1.95, 6.9, 1.09, 2.88, 1515}, - {1, 13.68, 1.83, 2.36, 17.2, 104, 2.42, 2.69, .42, 1.97, 3.84, 1.23, 2.87, 990}, - {1, 13.76, 1.53, 2.7, 19.5, 132, 2.95, 2.74, .5, 1.35, 5.4, 1.25, 3, 1235}, - {1, 13.51, 1.8, 2.65, 19, 110, 2.35, 2.53, .29, 1.54, 4.2, 1.1, 2.87, 1095}, - {1, 13.48, 1.81, 2.41, 20.5, 100, 2.7, 2.98, .26, 1.86, 5.1, 1.04, 3.47, 920}, - {1, 13.28, 1.64, 2.84, 15.5, 110, 2.6, 2.68, .34, 1.36, 4.6, 1.09, 2.78, 880}, - {1, 13.05, 1.65, 2.55, 18, 98, 2.45, 2.43, .29, 1.44, 4.25, 1.12, 2.51, 1105}, - {1, 13.07, 1.5, 2.1, 15.5, 98, 2.4, 2.64, .28, 1.37, 3.7, 1.18, 2.69, 1020}, - {1, 14.22, 3.99, 2.51, 13.2, 128, 3, 3.04, .2, 2.08, 5.1, .89, 3.53, 760}, - {1, 13.56, 1.71, 2.31, 16.2, 117, 3.15, 3.29, .34, 2.34, 6.13, .95, 3.38, 795}, - {1, 13.41, 3.84, 2.12, 18.8, 90, 2.45, 2.68, .27, 1.48, 4.28, .91, 3, 1035}, - {1, 13.88, 1.89, 2.59, 15, 101, 3.25, 3.56, .17, 1.7, 5.43, .88, 3.56, 1095}, - {1, 13.24, 3.98, 2.29, 17.5, 103, 2.64, 2.63, .32, 1.66, 4.36, .82, 3, 680}, - {1, 13.05, 1.77, 2.1, 17, 107, 3, 3, .28, 2.03, 5.04, .88, 3.35, 885}, - {1, 14.21, 4.04, 2.44, 18.9, 111, 2.85, 2.65, .3, 1.25, 5.24, .87, 3.33, 1080}, - {1, 14.38, 3.59, 2.28, 16, 102, 3.25, 3.17, .27, 2.19, 4.9, 1.04, 3.44, 1065}, - {1, 13.9, 1.68, 2.12, 16, 101, 3.1, 3.39, .21, 2.14, 6.1, .91, 3.33, 985}, - {1, 14.1, 2.02, 2.4, 18.8, 103, 2.75, 2.92, .32, 2.38, 6.2, 1.07, 2.75, 1060}, - {1, 13.94, 1.73, 2.27, 17.4, 108, 2.88, 3.54, .32, 2.08, 8.90, 1.12, 3.1, 1260}, - {1, 13.05, 1.73, 2.04, 12.4, 92, 2.72, 3.27, .17, 2.91, 7.2, 1.12, 2.91, 1150}, - {1, 13.83, 1.65, 2.6, 17.2, 94, 2.45, 2.99, .22, 2.29, 5.6, 1.24, 3.37, 1265}, - {1, 13.82, 1.75, 2.42, 14, 111, 3.88, 3.74, .32, 1.87, 7.05, 1.01, 3.26, 1190}, - {1, 13.77, 1.9, 2.68, 17.1, 115, 3, 2.79, .39, 1.68, 6.3, 1.13, 2.93, 1375}, - {1, 13.74, 1.67, 2.25, 16.4, 118, 2.6, 2.9, .21, 1.62, 5.85, .92, 3.2, 1060}, - {1, 13.56, 1.73, 2.46, 20.5, 116, 2.96, 2.78, .2, 2.45, 6.25, .98, 3.03, 1120}, - {1, 14.22, 1.7, 2.3, 16.3, 118, 3.2, 3, .26, 2.03, 6.38, .94, 3.31, 970}, - {1, 13.29, 1.97, 2.68, 16.8, 102, 3, 3.23, .31, 1.66, 6, 1.07, 2.84, 1270}, - {1, 13.72, 1.43, 2.5, 16.7, 108, 3.4, 3.67, .19, 2.04, 6.8, .89, 2.87, 1285}, - {2, 12.37, .94, 1.36, 10.6, 88, 1.98, .57, .28, .42, 1.95, 1.05, 1.82, 520}, - {2, 12.33, 1.1, 2.28, 16, 101, 2.05, 1.09, .63, .41, 3.27, 1.25, 1.67, 680}, - {2, 12.64, 1.36, 2.02, 16.8, 100, 2.02, 1.41, .53, .62, 5.75, .98, 1.59, 450}, - {2, 13.67, 1.25, 1.92, 18, 94, 2.1, 1.79, .32, .73, 3.8, 1.23, 2.46, 630}, - {2, 12.37, 1.13, 2.16, 19, 87, 3.5, 3.1, .19, 1.87, 4.45, 1.22, 2.87, 420}, - {2, 12.17, 1.45, 2.53, 19, 104, 1.89, 1.75, .45, 1.03, 2.95, 1.45, 2.23, 355}, - {2, 12.37, 1.21, 2.56, 18.1, 98, 2.42, 2.65, .37, 2.08, 4.6, 1.19, 2.3, 678}, - {2, 13.11, 1.01, 1.7, 15, 78, 2.98, 3.18, .26, 2.28, 5.3, 1.12, 3.18, 502}, - {2, 12.37, 1.17, 1.92, 19.6, 78, 2.11, 2, .27, 1.04, 4.68, 1.12, 3.48, 510}, - {2, 13.34, .94, 2.36, 17, 110, 2.53, 1.3, .55, .42, 3.17, 1.02, 1.93, 750}, - {2, 12.21, 1.19, 1.75, 16.8, 151, 1.85, 1.28, .14, 2.5, 2.85, 1.28, 3.07, 718}, - {2, 12.29, 1.61, 2.21, 20.4, 103, 1.1, 1.02, .37, 1.46, 3.05, .906, 1.82, 870}, - {2, 13.86, 1.51, 2.67, 25, 86, 2.95, 2.86, .21, 1.87, 3.38, 1.36, 3.16, 410}, - {2, 13.49, 1.66, 2.24, 24, 87, 1.88, 1.84, .27, 1.03, 3.74, .98, 2.78, 472}, - {2, 12.99, 1.67, 2.6, 30, 139, 3.3, 2.89, .21, 1.96, 3.35, 1.31, 3.5, 985}, - {2, 11.96, 1.09, 2.3, 21, 101, 3.38, 2.14, .13, 1.65, 3.21, .99, 3.13, 886}, - {2, 11.66, 1.88, 1.92, 16, 97, 1.61, 1.57, .34, 1.15, 3.8, 1.23, 2.14, 428}, - {2, 13.03, .9, 1.71, 16, 86, 1.95, 2.03, .24, 1.46, 4.6, 1.19, 2.48, 392}, - {2, 11.84, 2.89, 2.23, 18, 112, 1.72, 1.32, .43, .95, 2.65, .96, 2.52, 500}, - {2, 12.33, .99, 1.95, 14.8, 136, 1.9, 1.85, .35, 2.76, 3.4, 1.06, 2.31, 750}, - {2, 12.7, 3.87, 2.4, 23, 101, 2.83, 2.55, .43, 1.95, 2.57, 1.19, 3.13, 463}, - {2, 12, .92, 2, 19, 86, 2.42, 2.26, .3, 1.43, 2.5, 1.38, 3.12, 278}, - {2, 12.72, 1.81, 2.2, 18.8, 86, 2.2, 2.53, .26, 1.77, 3.9, 1.16, 3.14, 714}, - {2, 12.08, 1.13, 2.51, 24, 78, 2, 1.58, .4, 1.4, 2.2, 1.31, 2.72, 630}, - {2, 13.05, 3.86, 2.32, 22.5, 85, 1.65, 1.59, .61, 1.62, 4.8, .84, 2.01, 515}, - {2, 11.84, .89, 2.58, 18, 94, 2.2, 2.21, .22, 2.35, 3.05, .79, 3.08, 520}, - {2, 12.67, .98, 2.24, 18, 99, 2.2, 1.94, .3, 1.46, 2.62, 1.23, 3.16, 450}, - {2, 12.16, 1.61, 2.31, 22.8, 90, 1.78, 1.69, .43, 1.56, 2.45, 1.33, 2.26, 495}, - {2, 11.65, 1.67, 2.62, 26, 88, 1.92, 1.61, .4, 1.34, 2.6, 1.36, 3.21, 562}, - {2, 11.64, 2.06, 2.46, 21.6, 84, 1.95, 1.69, .48, 1.35, 2.8, 1, 2.75, 680}, - {2, 12.08, 1.33, 2.3, 23.6, 70, 2.2, 1.59, .42, 1.38, 1.74, 1.07, 3.21, 625}, - {2, 12.08, 1.83, 2.32, 18.5, 81, 1.6, 1.5, .52, 1.64, 2.4, 1.08, 2.27, 480}, - {2, 12, 1.51, 2.42, 22, 86, 1.45, 1.25, .5, 1.63, 3.6, 1.05, 2.65, 450}, - {2, 12.69, 1.53, 2.26, 20.7, 80, 1.38, 1.46, .58, 1.62, 3.05, .96, 2.06, 495}, - {2, 12.29, 2.83, 2.22, 18, 88, 2.45, 2.25, .25, 1.99, 2.15, 1.15, 3.3, 290}, - {2, 11.62, 1.99, 2.28, 18, 98, 3.02, 2.26, .17, 1.35, 3.25, 1.16, 2.96, 345}, - {2, 12.47, 1.52, 2.2, 19, 162, 2.5, 2.27, .32, 3.28, 2.6, 1.16, 2.63, 937}, - {2, 11.81, 2.12, 2.74, 21.5, 134, 1.6, .99, .14, 1.56, 2.5, .95, 2.26, 625}, - {2, 12.29, 1.41, 1.98, 16, 85, 2.55, 2.5, .29, 1.77, 2.9, 1.23, 2.74, 428}, - {2, 12.37, 1.07, 2.1, 18.5, 88, 3.52, 3.75, .24, 1.95, 4.5, 1.04, 2.77, 660}, - {2, 12.29, 3.17, 2.21, 18, 88, 2.85, 2.99, .45, 2.81, 2.3, 1.42, 2.83, 406}, - {2, 12.08, 2.08, 1.7, 17.5, 97, 2.23, 2.17, .26, 1.4, 3.3, 1.27, 2.96, 710}, - {2, 12.6, 1.34, 1.9, 18.5, 88, 1.45, 1.36, .29, 1.35, 2.45, 1.04, 2.77, 562}, - {2, 12.34, 2.45, 2.46, 21, 98, 2.56, 2.11, .34, 1.31, 2.8, .8, 3.38, 438}, - {2, 11.82, 1.72, 1.88, 19.5, 86, 2.5, 1.64, .37, 1.42, 2.06, .94, 2.44, 415}, - {2, 12.51, 1.73, 1.98, 20.5, 85, 2.2, 1.92, .32, 1.48, 2.94, 1.04, 3.57, 672}, - {2, 12.42, 2.55, 2.27, 22, 90, 1.68, 1.84, .66, 1.42, 2.7, .86, 3.3, 315}, - {2, 12.25, 1.73, 2.12, 19, 80, 1.65, 2.03, .37, 1.63, 3.4, 1, 3.17, 510}, - {2, 12.72, 1.75, 2.28, 22.5, 84, 1.38, 1.76, .48, 1.63, 3.3, .88, 2.42, 488}, - {2, 12.22, 1.29, 1.94, 19, 92, 2.36, 2.04, .39, 2.08, 2.7, .86, 3.02, 312}, - {2, 11.61, 1.35, 2.7, 20, 94, 2.74, 2.92, .29, 2.49, 2.65, .96, 3.26, 680}, - {2, 11.46, 3.74, 1.82, 19.5, 107, 3.18, 2.58, .24, 3.58, 2.9, .75, 2.81, 562}, - {2, 12.52, 2.43, 2.17, 21, 88, 2.55, 2.27, .26, 1.22, 2, .9, 2.78, 325}, - {2, 11.76, 2.68, 2.92, 20, 103, 1.75, 2.03, .6, 1.05, 3.8, 1.23, 2.5, 607}, - {2, 11.41, .74, 2.5, 21, 88, 2.48, 2.01, .42, 1.44, 3.08, 1.1, 2.31, 434}, - {2, 12.08, 1.39, 2.5, 22.5, 84, 2.56, 2.29, .43, 1.04, 2.9, .93, 3.19, 385}, - {2, 11.03, 1.51, 2.2, 21.5, 85, 2.46, 2.17, .52, 2.01, 1.9, 1.71, 2.87, 407}, - {2, 11.82, 1.47, 1.99, 20.8, 86, 1.98, 1.6, .3, 1.53, 1.95, .95, 3.33, 495}, - {2, 12.42, 1.61, 2.19, 22.5, 108, 2, 2.09, .34, 1.61, 2.06, 1.06, 2.96, 345}, - {2, 12.77, 3.43, 1.98, 16, 80, 1.63, 1.25, .43, .83, 3.4, .7, 2.12, 372}, - {2, 12, 3.43, 2, 19, 87, 2, 1.64, .37, 1.87, 1.28, .93, 3.05, 564}, - {2, 11.45, 2.4, 2.42, 20, 96, 2.9, 2.79, .32, 1.83, 3.25, .8, 3.39, 625}, - {2, 11.56, 2.05, 3.23, 28.5, 119, 3.18, 5.08, .47, 1.87, 6, .93, 3.69, 465}, - {2, 12.42, 4.43, 2.73, 26.5, 102, 2.2, 2.13, .43, 1.71, 2.08, .92, 3.12, 365}, - {2, 13.05, 5.8, 2.13, 21.5, 86, 2.62, 2.65, .3, 2.01, 2.6, .73, 3.1, 380}, - {2, 11.87, 4.31, 2.39, 21, 82, 2.86, 3.03, .21, 2.91, 2.8, .75, 3.64, 380}, - {2, 12.07, 2.16, 2.17, 21, 85, 2.6, 2.65, .37, 1.35, 2.76, .86, 3.28, 378}, - {2, 12.43, 1.53, 2.29, 21.5, 86, 2.74, 3.15, .39, 1.77, 3.94, .69, 2.84, 352}, - {2, 11.79, 2.13, 2.78, 28.5, 92, 2.13, 2.24, .58, 1.76, 3, .97, 2.44, 466}, - {2, 12.37, 1.63, 2.3, 24.5, 88, 2.22, 2.45, .4, 1.9, 2.12, .89, 2.78, 342}, - {2, 12.04, 4.3, 2.38, 22, 80, 2.1, 1.75, .42, 1.35, 2.6, .79, 2.57, 580}, - {3, 12.86, 1.35, 2.32, 18, 122, 1.51, 1.25, .21, .94, 4.1, .76, 1.29, 630}, - {3, 12.88, 2.99, 2.4, 20, 104, 1.3, 1.22, .24, .83, 5.4, .74, 1.42, 530}, - {3, 12.81, 2.31, 2.4, 24, 98, 1.15, 1.09, .27, .83, 5.7, .66, 1.36, 560}, - {3, 12.7, 3.55, 2.36, 21.5, 106, 1.7, 1.2, .17, .84, 5, .78, 1.29, 600}, - {3, 12.51, 1.24, 2.25, 17.5, 85, 2, .58, .6, 1.25, 5.45, .75, 1.51, 650}, - {3, 12.6, 2.46, 2.2, 18.5, 94, 1.62, .66, .63, .94, 7.1, .73, 1.58, 695}, - {3, 12.25, 4.72, 2.54, 21, 89, 1.38, .47, .53, .8, 3.85, .75, 1.27, 720}, - {3, 12.53, 5.51, 2.64, 25, 96, 1.79, .6, .63, 1.1, 5, .82, 1.69, 515}, - {3, 13.49, 3.59, 2.19, 19.5, 88, 1.62, .48, .58, .88, 5.7, .81, 1.82, 580}, - {3, 12.84, 2.96, 2.61, 24, 101, 2.32, .6, .53, .81, 4.92, .89, 2.15, 590}, - {3, 12.93, 2.81, 2.7, 21, 96, 1.54, .5, .53, .75, 4.6, .77, 2.31, 600}, - {3, 13.36, 2.56, 2.35, 20, 89, 1.4, .5, .37, .64, 5.6, .7, 2.47, 780}, - {3, 13.52, 3.17, 2.72, 23.5, 97, 1.55, .52, .5, .55, 4.35, .89, 2.06, 520}, - {3, 13.62, 4.95, 2.35, 20, 92, 2, .8, .47, 1.02, 4.4, .91, 2.05, 550}, - {3, 12.25, 3.88, 2.2, 18.5, 112, 1.38, .78, .29, 1.14, 8.21, .65, 2, 855}, - {3, 13.16, 3.57, 2.15, 21, 102, 1.5, .55, .43, 1.3, 4, .6, 1.68, 830}, - {3, 13.88, 5.04, 2.23, 20, 80, .98, .34, .4, .68, 4.9, .58, 1.33, 415}, - {3, 12.87, 4.61, 2.48, 21.5, 86, 1.7, .65, .47, .86, 7.65, .54, 1.86, 625}, - {3, 13.32, 3.24, 2.38, 21.5, 92, 1.93, .76, .45, 1.25, 8.42, .55, 1.62, 650}, - {3, 13.08, 3.9, 2.36, 21.5, 113, 1.41, 1.39, .34, 1.14, 9.40, .57, 1.33, 550}, - {3, 13.5, 3.12, 2.62, 24, 123, 1.4, 1.57, .22, 1.25, 8.60, .59, 1.3, 500}, - {3, 12.79, 2.67, 2.48, 22, 112, 1.48, 1.36, .24, 1.26, 10.8, .48, 1.47, 480}, - {3, 13.11, 1.9, 2.75, 25.5, 116, 2.2, 1.28, .26, 1.56, 7.1, .61, 1.33, 425}, - {3, 13.23, 3.3, 2.28, 18.5, 98, 1.8, .83, .61, 1.87, 10.52, .56, 1.51, 675}, - {3, 12.58, 1.29, 2.1, 20, 103, 1.48, .58, .53, 1.4, 7.6, .58, 1.55, 640}, - {3, 13.17, 5.19, 2.32, 22, 93, 1.74, .63, .61, 1.55, 7.9, .6, 1.48, 725}, - {3, 13.84, 4.12, 2.38, 19.5, 89, 1.8, .83, .48, 1.56, 9.01, .57, 1.64, 480}, - {3, 12.45, 3.03, 2.64, 27, 97, 1.9, .58, .63, 1.14, 7.5, .67, 1.73, 880}, - {3, 14.34, 1.68, 2.7, 25, 98, 2.8, 1.31, .53, 2.7, 13, .57, 1.96, 660}, - {3, 13.48, 1.67, 2.64, 22.5, 89, 2.6, 1.1, .52, 2.29, 11.75, .57, 1.78, 620}, - {3, 12.36, 3.83, 2.38, 21, 88, 2.3, .92, .5, 1.04, 7.65, .56, 1.58, 520}, - {3, 13.69, 3.26, 2.54, 20, 107, 1.83, .56, .5, .8, 5.88, .96, 1.82, 680}, - {3, 12.85, 3.27, 2.58, 22, 106, 1.65, .6, .6, .96, 5.58, .87, 2.11, 570}, - {3, 12.96, 3.45, 2.35, 18.5, 106, 1.39, .7, .4, .94, 5.28, .68, 1.75, 675}, - {3, 13.78, 2.76, 2.3, 22, 90, 1.35, .68, .41, 1.03, 9.58, .7, 1.68, 615}, - {3, 13.73, 4.36, 2.26, 22.5, 88, 1.28, .47, .52, 1.15, 6.62, .78, 1.75, 520}, - {3, 13.45, 3.7, 2.6, 23, 111, 1.7, .92, .43, 1.46, 10.68, .85, 1.56, 695}, - {3, 12.82, 3.37, 2.3, 19.5, 88, 1.48, .66, .4, .97, 10.26, .72, 1.75, 685}, - {3, 13.58, 2.58, 2.69, 24.5, 105, 1.55, .84, .39, 1.54, 8.66, .74, 1.8, 750}, - {3, 13.4, 4.6, 2.86, 25, 112, 1.98, .96, .27, 1.11, 8.5, .67, 1.92, 630}, - {3, 12.2, 3.03, 2.32, 19, 96, 1.25, .49, .4, .73, 5.5, .66, 1.83, 510}, - {3, 12.77, 2.39, 2.28, 19.5, 86, 1.39, .51, .48, .64, 9.899999, .57, 1.63, 470}, - {3, 14.16, 2.51, 2.48, 20, 91, 1.68, .7, .44, 1.24, 9.7, .62, 1.71, 660}, - {3, 13.71, 5.65, 2.45, 20.5, 95, 1.68, .61, .52, 1.06, 7.7, .64, 1.74, 740}, - {3, 13.4, 3.91, 2.48, 23, 102, 1.8, .75, .43, 1.41, 7.3, .7, 1.56, 750}, - {3, 13.27, 4.28, 2.26, 20, 120, 1.59, .69, .43, 1.35, 10.2, .59, 1.56, 835}, - {3, 13.17, 2.59, 2.37, 20, 120, 1.65, .68, .53, 1.46, 9.3, .6, 1.62, 840}, - {3, 14.13, 4.1, 2.74, 24.5, 96, 2.05, .76, .56, 1.35, 9.2, .61, 1.6, 560} + {1, 14.23, 1.71, 2.43, 15.6, 127, 2.8, 3.06, .28, 2.29, 5.64, 1.04, 3.92, 1065}, + {1, 13.2, 1.78, 2.14, 11.2, 100, 2.65, 2.76, .26, 1.28, 4.38, 1.05, 3.4, 1050}, + {1, 13.16, 2.36, 2.67, 18.6, 101, 2.8, 3.24, .3, 2.81, 5.68, 1.03, 3.17, 1185}, + {1, 14.37, 1.95, 2.5, 16.8, 113, 3.85, 3.49, .24, 2.18, 7.8, .86, 3.45, 1480}, + {1, 13.24, 2.59, 2.87, 21, 118, 2.8, 2.69, .39, 1.82, 4.32, 1.04, 2.93, 735}, + {1, 14.2, 1.76, 2.45, 15.2, 112, 3.27, 3.39, .34, 1.97, 6.75, 1.05, 2.85, 1450}, + {1, 14.39, 1.87, 2.45, 14.6, 96, 2.5, 2.52, .3, 1.98, 5.25, 1.02, 3.58, 1290}, + {1, 14.06, 2.15, 2.61, 17.6, 121, 2.6, 2.51, .31, 1.25, 5.05, 1.06, 3.58, 1295}, + {1, 14.83, 1.64, 2.17, 14, 97, 2.8, 2.98, .29, 1.98, 5.2, 1.08, 2.85, 1045}, + {1, 13.86, 1.35, 2.27, 16, 98, 2.98, 3.15, .22, 1.85, 7.22, 1.01, 3.55, 1045}, + {1, 14.1, 2.16, 2.3, 18, 105, 2.95, 3.32, .22, 2.38, 5.75, 1.25, 3.17, 1510}, + {1, 14.12, 1.48, 2.32, 16.8, 95, 2.2, 2.43, .26, 1.57, 5, 1.17, 2.82, 1280}, + {1, 13.75, 1.73, 2.41, 16, 89, 2.6, 2.76, .29, 1.81, 5.6, 1.15, 2.9, 1320}, + {1, 14.75, 1.73, 2.39, 11.4, 91, 3.1, 3.69, .43, 2.81, 5.4, 1.25, 2.73, 1150}, + {1, 14.38, 1.87, 2.38, 12, 102, 3.3, 3.64, .29, 2.96, 7.5, 1.2, 3, 1547}, + {1, 13.63, 1.81, 2.7, 17.2, 112, 2.85, 2.91, .3, 1.46, 7.3, 1.28, 2.88, 1310}, + {1, 14.3, 1.92, 2.72, 20, 120, 2.8, 3.14, .33, 1.97, 6.2, 1.07, 2.65, 1280}, + {1, 13.83, 1.57, 2.62, 20, 115, 2.95, 3.4, .4, 1.72, 6.6, 1.13, 2.57, 1130}, + {1, 14.19, 1.59, 2.48, 16.5, 108, 3.3, 3.93, .32, 1.86, 8.7, 1.23, 2.82, 1680}, + {1, 13.64, 3.1, 2.56, 15.2, 116, 2.7, 3.03, .17, 1.66, 5.1, .96, 3.36, 845}, + {1, 14.06, 1.63, 2.28, 16, 126, 3, 3.17, .24, 2.1, 5.65, 1.09, 3.71, 780}, + {1, 12.93, 3.8, 2.65, 18.6, 102, 2.41, 2.41, .25, 1.98, 4.5, 1.03, 3.52, 770}, + {1, 13.71, 1.86, 2.36, 16.6, 101, 2.61, 2.88, .27, 1.69, 3.8, 1.11, 4, 1035}, + {1, 12.85, 1.6, 2.52, 17.8, 95, 2.48, 2.37, .26, 1.46, 3.93, 1.09, 3.63, 1015}, + {1, 13.5, 1.81, 2.61, 20, 96, 2.53, 2.61, .28, 1.66, 3.52, 1.12, 3.82, 845}, + {1, 13.05, 2.05, 3.22, 25, 124, 2.63, 2.68, .47, 1.92, 3.58, 1.13, 3.2, 830}, + {1, 13.39, 1.77, 2.62, 16.1, 93, 2.85, 2.94, .34, 1.45, 4.8, .92, 3.22, 1195}, + {1, 13.3, 1.72, 2.14, 17, 94, 2.4, 2.19, .27, 1.35, 3.95, 1.02, 2.77, 1285}, + {1, 13.87, 1.9, 2.8, 19.4, 107, 2.95, 2.97, .37, 1.76, 4.5, 1.25, 3.4, 915}, + {1, 14.02, 1.68, 2.21, 16, 96, 2.65, 2.33, .26, 1.98, 4.7, 1.04, 3.59, 1035}, + {1, 13.73, 1.5, 2.7, 22.5, 101, 3, 3.25, .29, 2.38, 5.7, 1.19, 2.71, 1285}, + {1, 13.58, 1.66, 2.36, 19.1, 106, 2.86, 3.19, .22, 1.95, 6.9, 1.09, 2.88, 1515}, + {1, 13.68, 1.83, 2.36, 17.2, 104, 2.42, 2.69, .42, 1.97, 3.84, 1.23, 2.87, 990}, + {1, 13.76, 1.53, 2.7, 19.5, 132, 2.95, 2.74, .5, 1.35, 5.4, 1.25, 3, 1235}, + {1, 13.51, 1.8, 2.65, 19, 110, 2.35, 2.53, .29, 1.54, 4.2, 1.1, 2.87, 1095}, + {1, 13.48, 1.81, 2.41, 20.5, 100, 2.7, 2.98, .26, 1.86, 5.1, 1.04, 3.47, 920}, + {1, 13.28, 1.64, 2.84, 15.5, 110, 2.6, 2.68, .34, 1.36, 4.6, 1.09, 2.78, 880}, + {1, 13.05, 1.65, 2.55, 18, 98, 2.45, 2.43, .29, 1.44, 4.25, 1.12, 2.51, 1105}, + {1, 13.07, 1.5, 2.1, 15.5, 98, 2.4, 2.64, .28, 1.37, 3.7, 1.18, 2.69, 1020}, + {1, 14.22, 3.99, 2.51, 13.2, 128, 3, 3.04, .2, 2.08, 5.1, .89, 3.53, 760}, + {1, 13.56, 1.71, 2.31, 16.2, 117, 3.15, 3.29, .34, 2.34, 6.13, .95, 3.38, 795}, + {1, 13.41, 3.84, 2.12, 18.8, 90, 2.45, 2.68, .27, 1.48, 4.28, .91, 3, 1035}, + {1, 13.88, 1.89, 2.59, 15, 101, 3.25, 3.56, .17, 1.7, 5.43, .88, 3.56, 1095}, + {1, 13.24, 3.98, 2.29, 17.5, 103, 2.64, 2.63, .32, 1.66, 4.36, .82, 3, 680}, + {1, 13.05, 1.77, 2.1, 17, 107, 3, 3, .28, 2.03, 5.04, .88, 3.35, 885}, + {1, 14.21, 4.04, 2.44, 18.9, 111, 2.85, 2.65, .3, 1.25, 5.24, .87, 3.33, 1080}, + {1, 14.38, 3.59, 2.28, 16, 102, 3.25, 3.17, .27, 2.19, 4.9, 1.04, 3.44, 1065}, + {1, 13.9, 1.68, 2.12, 16, 101, 3.1, 3.39, .21, 2.14, 6.1, .91, 3.33, 985}, + {1, 14.1, 2.02, 2.4, 18.8, 103, 2.75, 2.92, .32, 2.38, 6.2, 1.07, 2.75, 1060}, + {1, 13.94, 1.73, 2.27, 17.4, 108, 2.88, 3.54, .32, 2.08, 8.90, 1.12, 3.1, 1260}, + {1, 13.05, 1.73, 2.04, 12.4, 92, 2.72, 3.27, .17, 2.91, 7.2, 1.12, 2.91, 1150}, + {1, 13.83, 1.65, 2.6, 17.2, 94, 2.45, 2.99, .22, 2.29, 5.6, 1.24, 3.37, 1265}, + {1, 13.82, 1.75, 2.42, 14, 111, 3.88, 3.74, .32, 1.87, 7.05, 1.01, 3.26, 1190}, + {1, 13.77, 1.9, 2.68, 17.1, 115, 3, 2.79, .39, 1.68, 6.3, 1.13, 2.93, 1375}, + {1, 13.74, 1.67, 2.25, 16.4, 118, 2.6, 2.9, .21, 1.62, 5.85, .92, 3.2, 1060}, + {1, 13.56, 1.73, 2.46, 20.5, 116, 2.96, 2.78, .2, 2.45, 6.25, .98, 3.03, 1120}, + {1, 14.22, 1.7, 2.3, 16.3, 118, 3.2, 3, .26, 2.03, 6.38, .94, 3.31, 970}, + {1, 13.29, 1.97, 2.68, 16.8, 102, 3, 3.23, .31, 1.66, 6, 1.07, 2.84, 1270}, + {1, 13.72, 1.43, 2.5, 16.7, 108, 3.4, 3.67, .19, 2.04, 6.8, .89, 2.87, 1285}, + {2, 12.37, .94, 1.36, 10.6, 88, 1.98, .57, .28, .42, 1.95, 1.05, 1.82, 520}, + {2, 12.33, 1.1, 2.28, 16, 101, 2.05, 1.09, .63, .41, 3.27, 1.25, 1.67, 680}, + {2, 12.64, 1.36, 2.02, 16.8, 100, 2.02, 1.41, .53, .62, 5.75, .98, 1.59, 450}, + {2, 13.67, 1.25, 1.92, 18, 94, 2.1, 1.79, .32, .73, 3.8, 1.23, 2.46, 630}, + {2, 12.37, 1.13, 2.16, 19, 87, 3.5, 3.1, .19, 1.87, 4.45, 1.22, 2.87, 420}, + {2, 12.17, 1.45, 2.53, 19, 104, 1.89, 1.75, .45, 1.03, 2.95, 1.45, 2.23, 355}, + {2, 12.37, 1.21, 2.56, 18.1, 98, 2.42, 2.65, .37, 2.08, 4.6, 1.19, 2.3, 678}, + {2, 13.11, 1.01, 1.7, 15, 78, 2.98, 3.18, .26, 2.28, 5.3, 1.12, 3.18, 502}, + {2, 12.37, 1.17, 1.92, 19.6, 78, 2.11, 2, .27, 1.04, 4.68, 1.12, 3.48, 510}, + {2, 13.34, .94, 2.36, 17, 110, 2.53, 1.3, .55, .42, 3.17, 1.02, 1.93, 750}, + {2, 12.21, 1.19, 1.75, 16.8, 151, 1.85, 1.28, .14, 2.5, 2.85, 1.28, 3.07, 718}, + {2, 12.29, 1.61, 2.21, 20.4, 103, 1.1, 1.02, .37, 1.46, 3.05, .906, 1.82, 870}, + {2, 13.86, 1.51, 2.67, 25, 86, 2.95, 2.86, .21, 1.87, 3.38, 1.36, 3.16, 410}, + {2, 13.49, 1.66, 2.24, 24, 87, 1.88, 1.84, .27, 1.03, 3.74, .98, 2.78, 472}, + {2, 12.99, 1.67, 2.6, 30, 139, 3.3, 2.89, .21, 1.96, 3.35, 1.31, 3.5, 985}, + {2, 11.96, 1.09, 2.3, 21, 101, 3.38, 2.14, .13, 1.65, 3.21, .99, 3.13, 886}, + {2, 11.66, 1.88, 1.92, 16, 97, 1.61, 1.57, .34, 1.15, 3.8, 1.23, 2.14, 428}, + {2, 13.03, .9, 1.71, 16, 86, 1.95, 2.03, .24, 1.46, 4.6, 1.19, 2.48, 392}, + {2, 11.84, 2.89, 2.23, 18, 112, 1.72, 1.32, .43, .95, 2.65, .96, 2.52, 500}, + {2, 12.33, .99, 1.95, 14.8, 136, 1.9, 1.85, .35, 2.76, 3.4, 1.06, 2.31, 750}, + {2, 12.7, 3.87, 2.4, 23, 101, 2.83, 2.55, .43, 1.95, 2.57, 1.19, 3.13, 463}, + {2, 12, .92, 2, 19, 86, 2.42, 2.26, .3, 1.43, 2.5, 1.38, 3.12, 278}, + {2, 12.72, 1.81, 2.2, 18.8, 86, 2.2, 2.53, .26, 1.77, 3.9, 1.16, 3.14, 714}, + {2, 12.08, 1.13, 2.51, 24, 78, 2, 1.58, .4, 1.4, 2.2, 1.31, 2.72, 630}, + {2, 13.05, 3.86, 2.32, 22.5, 85, 1.65, 1.59, .61, 1.62, 4.8, .84, 2.01, 515}, + {2, 11.84, .89, 2.58, 18, 94, 2.2, 2.21, .22, 2.35, 3.05, .79, 3.08, 520}, + {2, 12.67, .98, 2.24, 18, 99, 2.2, 1.94, .3, 1.46, 2.62, 1.23, 3.16, 450}, + {2, 12.16, 1.61, 2.31, 22.8, 90, 1.78, 1.69, .43, 1.56, 2.45, 1.33, 2.26, 495}, + {2, 11.65, 1.67, 2.62, 26, 88, 1.92, 1.61, .4, 1.34, 2.6, 1.36, 3.21, 562}, + {2, 11.64, 2.06, 2.46, 21.6, 84, 1.95, 1.69, .48, 1.35, 2.8, 1, 2.75, 680}, + {2, 12.08, 1.33, 2.3, 23.6, 70, 2.2, 1.59, .42, 1.38, 1.74, 1.07, 3.21, 625}, + {2, 12.08, 1.83, 2.32, 18.5, 81, 1.6, 1.5, .52, 1.64, 2.4, 1.08, 2.27, 480}, + {2, 12, 1.51, 2.42, 22, 86, 1.45, 1.25, .5, 1.63, 3.6, 1.05, 2.65, 450}, + {2, 12.69, 1.53, 2.26, 20.7, 80, 1.38, 1.46, .58, 1.62, 3.05, .96, 2.06, 495}, + {2, 12.29, 2.83, 2.22, 18, 88, 2.45, 2.25, .25, 1.99, 2.15, 1.15, 3.3, 290}, + {2, 11.62, 1.99, 2.28, 18, 98, 3.02, 2.26, .17, 1.35, 3.25, 1.16, 2.96, 345}, + {2, 12.47, 1.52, 2.2, 19, 162, 2.5, 2.27, .32, 3.28, 2.6, 1.16, 2.63, 937}, + {2, 11.81, 2.12, 2.74, 21.5, 134, 1.6, .99, .14, 1.56, 2.5, .95, 2.26, 625}, + {2, 12.29, 1.41, 1.98, 16, 85, 2.55, 2.5, .29, 1.77, 2.9, 1.23, 2.74, 428}, + {2, 12.37, 1.07, 2.1, 18.5, 88, 3.52, 3.75, .24, 1.95, 4.5, 1.04, 2.77, 660}, + {2, 12.29, 3.17, 2.21, 18, 88, 2.85, 2.99, .45, 2.81, 2.3, 1.42, 2.83, 406}, + {2, 12.08, 2.08, 1.7, 17.5, 97, 2.23, 2.17, .26, 1.4, 3.3, 1.27, 2.96, 710}, + {2, 12.6, 1.34, 1.9, 18.5, 88, 1.45, 1.36, .29, 1.35, 2.45, 1.04, 2.77, 562}, + {2, 12.34, 2.45, 2.46, 21, 98, 2.56, 2.11, .34, 1.31, 2.8, .8, 3.38, 438}, + {2, 11.82, 1.72, 1.88, 19.5, 86, 2.5, 1.64, .37, 1.42, 2.06, .94, 2.44, 415}, + {2, 12.51, 1.73, 1.98, 20.5, 85, 2.2, 1.92, .32, 1.48, 2.94, 1.04, 3.57, 672}, + {2, 12.42, 2.55, 2.27, 22, 90, 1.68, 1.84, .66, 1.42, 2.7, .86, 3.3, 315}, + {2, 12.25, 1.73, 2.12, 19, 80, 1.65, 2.03, .37, 1.63, 3.4, 1, 3.17, 510}, + {2, 12.72, 1.75, 2.28, 22.5, 84, 1.38, 1.76, .48, 1.63, 3.3, .88, 2.42, 488}, + {2, 12.22, 1.29, 1.94, 19, 92, 2.36, 2.04, .39, 2.08, 2.7, .86, 3.02, 312}, + {2, 11.61, 1.35, 2.7, 20, 94, 2.74, 2.92, .29, 2.49, 2.65, .96, 3.26, 680}, + {2, 11.46, 3.74, 1.82, 19.5, 107, 3.18, 2.58, .24, 3.58, 2.9, .75, 2.81, 562}, + {2, 12.52, 2.43, 2.17, 21, 88, 2.55, 2.27, .26, 1.22, 2, .9, 2.78, 325}, + {2, 11.76, 2.68, 2.92, 20, 103, 1.75, 2.03, .6, 1.05, 3.8, 1.23, 2.5, 607}, + {2, 11.41, .74, 2.5, 21, 88, 2.48, 2.01, .42, 1.44, 3.08, 1.1, 2.31, 434}, + {2, 12.08, 1.39, 2.5, 22.5, 84, 2.56, 2.29, .43, 1.04, 2.9, .93, 3.19, 385}, + {2, 11.03, 1.51, 2.2, 21.5, 85, 2.46, 2.17, .52, 2.01, 1.9, 1.71, 2.87, 407}, + {2, 11.82, 1.47, 1.99, 20.8, 86, 1.98, 1.6, .3, 1.53, 1.95, .95, 3.3423, 495}, + {2, 12.42, 1.61, 2.19, 22.5, 108, 2, 2.09, .34, 1.61, 2.06, 1.06, 2.96, 345}, + {2, 12.77, 3.43, 1.98, 16, 80, 1.63, 1.25, .43, .83, 3.4, .7, 2.12, 372}, + {2, 12, 3.43, 2, 19, 87, 2, 1.64, .37, 1.87, 1.28, .93, 3.05, 564}, + {2, 11.45, 2.4, 2.42, 20, 96, 2.9, 2.79, .32, 1.83, 3.25, .8, 3.39, 625}, + {2, 11.56, 2.05, 3.23, 28.5, 119, 3.18, 5.08, .47, 1.87, 6, .93, 3.69, 465}, + {2, 12.42, 4.43, 2.73, 26.5, 102, 2.2, 2.13, .43, 1.71, 2.08, .92, 3.12, 365}, + {2, 13.05, 5.8, 2.13, 21.5, 86, 2.62, 2.65, .3, 2.01, 2.6, .73, 3.1, 380}, + {2, 11.87, 4.31, 2.39, 21, 82, 2.86, 3.03, .21, 2.91, 2.8, .75, 3.64, 380}, + {2, 12.07, 2.16, 2.17, 21, 85, 2.6, 2.65, .37, 1.35, 2.76, .86, 3.28, 378}, + {2, 12.43, 1.53, 2.29, 21.5, 86, 2.74, 3.15, .39, 1.77, 3.94, .69, 2.84, 352}, + {2, 11.79, 2.13, 2.78, 28.5, 92, 2.13, 2.24, .58, 1.76, 3, .97, 2.44, 466}, + {2, 12.37, 1.63, 2.3, 24.5, 88, 2.22, 2.45, .4, 1.9, 2.12, .89, 2.78, 342}, + {2, 12.04, 4.3, 2.38, 22, 80, 2.1, 1.75, .42, 1.35, 2.6, .79, 2.57, 580}, + {3, 12.86, 1.35, 2.32, 18, 122, 1.51, 1.25, .21, .94, 4.1, .76, 1.29, 630}, + {3, 12.88, 2.99, 2.4, 20, 104, 1.3, 1.22, .24, .83, 5.4, .74, 1.42, 530}, + {3, 12.81, 2.31, 2.4, 24, 98, 1.15, 1.09, .27, .83, 5.7, .66, 1.36, 560}, + {3, 12.7, 3.55, 2.36, 21.5, 106, 1.7, 1.2, .17, .84, 5, .78, 1.29, 600}, + {3, 12.51, 1.24, 2.25, 17.5, 85, 2, .58, .6, 1.25, 5.45, .75, 1.51, 650}, + {3, 12.6, 2.46, 2.2, 18.5, 94, 1.62, .66, .63, .94, 7.1, .73, 1.58, 695}, + {3, 12.25, 4.72, 2.54, 21, 89, 1.38, .47, .53, .8, 3.85, .75, 1.27, 720}, + {3, 12.53, 5.51, 2.64, 25, 96, 1.79, .6, .63, 1.1, 5, .82, 1.69, 515}, + {3, 13.49, 3.59, 2.19, 19.5, 88, 1.62, .48, .58, .88, 5.7, .81, 1.82, 580}, + {3, 12.84, 2.96, 2.61, 24, 101, 2.32, .6, .53, .81, 4.92, .89, 2.15, 590}, + {3, 12.93, 2.81, 2.7, 21, 96, 1.54, .5, .53, .75, 4.6, .77, 2.31, 600}, + {3, 13.36, 2.56, 2.35, 20, 89, 1.4, .5, .37, .64, 5.6, .7, 2.47, 780}, + {3, 13.52, 3.17, 2.72, 23.5, 97, 1.55, .52, .5, .55, 4.35, .89, 2.06, 520}, + {3, 13.62, 4.95, 2.35, 20, 92, 2, .8, .47, 1.02, 4.4, .91, 2.05, 550}, + {3, 12.25, 3.88, 2.2, 18.5, 112, 1.38, .78, .29, 1.14, 8.21, .65, 2, 855}, + {3, 13.16, 3.57, 2.15, 21, 102, 1.5, .55, .43, 1.3, 4, .6, 1.68, 830}, + {3, 13.88, 5.04, 2.23, 20, 80, .98, .34, .4, .68, 4.9, .58, 1.33, 415}, + {3, 12.87, 4.61, 2.48, 21.5, 86, 1.7, .65, .47, .86, 7.65, .54, 1.86, 625}, + {3, 13.32, 3.24, 2.38, 21.5, 92, 1.93, .76, .45, 1.25, 8.42, .55, 1.62, 650}, + {3, 13.08, 3.9, 2.36, 21.5, 113, 1.41, 1.39, .34, 1.14, 9.40, .57, 1.33, 550}, + {3, 13.5, 3.12, 2.62, 24, 123, 1.4, 1.57, .22, 1.25, 8.60, .59, 1.3, 500}, + {3, 12.79, 2.67, 2.48, 22, 112, 1.48, 1.36, .24, 1.26, 10.8, .48, 1.47, 480}, + {3, 13.11, 1.9, 2.75, 25.5, 116, 2.2, 1.28, .26, 1.56, 7.1, .61, 1.33, 425}, + {3, 13.23, 3.3, 2.28, 18.5, 98, 1.8, .83, .61, 1.87, 10.52, .56, 1.51, 675}, + {3, 12.58, 1.29, 2.1, 20, 103, 1.48, .58, .53, 1.4, 7.6, .58, 1.55, 640}, + {3, 13.17, 5.19, 2.32, 22, 93, 1.74, .63, .61, 1.55, 7.9, .6, 1.48, 725}, + {3, 13.84, 4.12, 2.38, 19.5, 89, 1.8, .83, .48, 1.56, 9.01, .57, 1.64, 480}, + {3, 12.45, 3.03, 2.64, 27, 97, 1.9, .58, .63, 1.14, 7.5, .67, 1.73, 880}, + {3, 14.34, 1.68, 2.7, 25, 98, 2.8, 1.31, .53, 2.7, 13, .57, 1.96, 660}, + {3, 13.48, 1.67, 2.64, 22.5, 89, 2.6, 1.1, .52, 2.29, 11.75, .57, 1.78, 620}, + {3, 12.36, 3.83, 2.38, 21, 88, 2.3, .92, .5, 1.04, 7.65, .56, 1.58, 520}, + {3, 13.69, 3.26, 2.54, 20, 107, 1.83, .56, .5, .8, 5.88, .96, 1.82, 680}, + {3, 12.85, 3.27, 2.58, 22, 106, 1.65, .6, .6, .96, 5.58, .87, 2.11, 570}, + {3, 12.96, 3.45, 2.35, 18.5, 106, 1.39, .7, .4, .94, 5.28, .68, 1.75, 675}, + {3, 13.78, 2.76, 2.3, 22, 90, 1.35, .68, .41, 1.03, 9.58, .7, 1.68, 615}, + {3, 13.73, 4.36, 2.26, 22.5, 88, 1.28, .47, .52, 1.15, 6.62, .78, 1.75, 520}, + {3, 13.45, 3.7, 2.6, 23, 111, 1.7, .92, .43, 1.46, 10.68, .85, 1.56, 695}, + {3, 12.82, 3.37, 2.3, 19.5, 88, 1.48, .66, .4, .97, 10.26, .72, 1.75, 685}, + {3, 13.58, 2.58, 2.69, 24.5, 105, 1.55, .84, .39, 1.54, 8.66, .74, 1.8, 750}, + {3, 13.4, 4.6, 2.86, 25, 112, 1.98, .96, .27, 1.11, 8.5, .67, 1.92, 630}, + {3, 12.2, 3.03, 2.32, 19, 96, 1.25, .49, .4, .73, 5.5, .66, 1.83, 510}, + {3, 12.77, 2.39, 2.28, 19.5, 86, 1.39, .51, .48, .64, 9.899999, .57, 1.63, 470}, + {3, 14.16, 2.51, 2.48, 20, 91, 1.68, .7, .44, 1.24, 9.7, .62, 1.71, 660}, + {3, 13.71, 5.65, 2.45, 20.5, 95, 1.68, .61, .52, 1.06, 7.7, .64, 1.74, 740}, + {3, 13.4, 3.91, 2.48, 23, 102, 1.8, .75, .43, 1.41, 7.3, .7, 1.56, 750}, + {3, 13.27, 4.28, 2.26, 20, 120, 1.59, .69, .43, 1.35, 10.2, .59, 1.56, 835}, + {3, 13.17, 2.59, 2.37, 20, 120, 1.65, .68, .53, 1.46, 9.3, .6, 1.62, 840}, + {3, 14.13, 4.1, 2.74, 24.5, 96, 2.05, .76, .56, 1.35, 9.2, .61, 1.6, 560} }; } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java index b83d526e29cd7..96a87c2ebbe03 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java @@ -18,6 +18,9 @@ package org.apache.ignite.examples.ml.tree.randomforest; import java.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -26,17 +29,15 @@ import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.examples.ml.util.TestCache; import org.apache.ignite.ml.composition.ModelsComposition; -import org.apache.ignite.ml.environment.LearningEnvironment; -import org.apache.ignite.ml.environment.logging.ConsoleLogger; -import org.apache.ignite.ml.environment.logging.MLLogger; -import org.apache.ignite.ml.environment.parallelism.ParallelismStrategy; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.tree.randomforest.RandomForestRegressionTrainer; import org.apache.ignite.ml.tree.randomforest.RandomForestTrainer; +import org.apache.ignite.ml.tree.randomforest.data.FeaturesCountSelectionStrategies; import org.apache.ignite.thread.IgniteThread; /** - * Example represents a solution for the task of price predictions for houses in Boston based on RandomForest + * Example represents a solution for the task of price predictions for houses in Boston based on RandomForestTrainer * implementation for regression. It shows an initialization of {@link RandomForestTrainer}, +initialization of Ignite * Cache, learning step and evaluation of model quality in terms of Mean Squared Error (MSE) and Mean Absolute Error * (MAE). @@ -60,12 +61,15 @@ public static void main(String[] args) throws InterruptedException { RandomForestRegressionExample.class.getSimpleName(), () -> { IgniteCache dataCache = new TestCache(ignite).fillCacheWith(data); - RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer(13, 4, 101, 0.3, 2, 0); - trainer.setEnvironment(LearningEnvironment.builder() - .withParallelismStrategy(ParallelismStrategy.Type.ON_DEFAULT_POOL) - .withLoggingFactory(ConsoleLogger.factory(MLLogger.VerboseLevel.LOW)) - .build() - ); + AtomicInteger indx = new AtomicInteger(0); + RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer( + IntStream.range(0, data[0].length - 1).mapToObj(x -> new FeatureMeta("", indx.getAndIncrement(), false)).collect(Collectors.toList()) + ).withCountOfTrees(101) + .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) + .withMaxDepth(4) + .withMinImpurityDelta(0.) + .withSubsampleSize(0.3) + .withSeed(0); ModelsComposition randomForest = trainer.fit(ignite, dataCache, (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), diff --git a/modules/ml/pom.xml b/modules/ml/pom.xml index fac5ab42c6429..ad31da256645d 100644 --- a/modules/ml/pom.xml +++ b/modules/ml/pom.xml @@ -21,6 +21,9 @@ --> 4.0.0 + + 3.6.1 + ignite-parent @@ -47,7 +50,7 @@ test-jar test - + org.apache.ignite ignite-indexing @@ -60,7 +63,7 @@ ignite-spring ${project.version} - + it.unimi.dsi fastutil @@ -118,6 +121,12 @@ ${mockito.version} test + + + org.apache.commons + commons-math3 + ${commons.math.version} + diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java index c37fdf7b82742..8a369ad1d0567 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java @@ -20,7 +20,7 @@ import org.apache.ignite.internal.util.typedef.internal.A; /** - * Predictions aggregator returning weighted sum of predictions. + * Predictions aggregator returning weighted plus of predictions. * result(p1, ..., pn) = bias + p1*w1 + ... + pn*wn */ public class WeightedPredictionsAggregator implements PredictionsAggregator { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/BucketMeta.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/BucketMeta.java new file mode 100644 index 0000000000000..4ac9adbfb45ad --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/BucketMeta.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.ml.dataset.feature; + +import java.io.Serializable; + +/** + * Bucket meta-information for feature histogram. + */ +public class BucketMeta implements Serializable { + /** Feature meta. */ + private final FeatureMeta featureMeta; + + /** Bucket size. */ + private double bucketSize; + + /** Min value of feature. */ + private double minVal; + + /** + * Creates an instance of BucketMeta. + * + * @param featureMeta Feature meta. + */ + public BucketMeta(FeatureMeta featureMeta) { + this.featureMeta = featureMeta; + } + + /** + * Returns bucket id for feature value. + * + * @param val Value. + * @return bucket id. + */ + public int getBucketId(Double val) { + if(featureMeta.isCategoricalFeature()) + return (int) Math.rint(val); + + return (int) Math.rint((val - minVal) / bucketSize); + } + + /** + * Returns mean value by bucket id. + * + * @param bucketId Bucket id. + * @return mean value of feature. + */ + public double bucketIdToValue(int bucketId) { + if(featureMeta.isCategoricalFeature()) + return (double) bucketId; + + return minVal + (bucketId + 0.5) * bucketSize; + } + + /** + * @param minVal Min value. + */ + public void setMinVal(double minVal) { + this.minVal = minVal; + } + + /** + * @param bucketSize Bucket size. + */ + public void setBucketSize(double bucketSize) { + this.bucketSize = bucketSize; + } + + /** */ + public FeatureMeta getFeatureMeta() { + return featureMeta; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/DistributionComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/DistributionComputer.java new file mode 100644 index 0000000000000..7582da0ba077f --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/DistributionComputer.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.dataset.feature; + +import java.util.TreeMap; + +/** + * Interface specifies an object that can compute some discrete distribution. + */ +public interface DistributionComputer { + /** + * Compute distribution function. + * + * @return Map represents discrete distribution function. + */ + public TreeMap computeDistributionFunction(); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/FeatureMeta.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/FeatureMeta.java new file mode 100644 index 0000000000000..a7d846f93c8ae --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/FeatureMeta.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.dataset.feature; + +import java.io.Serializable; + +/** + * Feature meta class. + */ +public class FeatureMeta implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = -2990950807063111877L; + + /** Name. */ + private final String name; + + /** Id of feature in feature vector. */ + private final int featureId; + + /** Is categorical feature flag. */ + private final boolean isCategoricalFeature; + + /** + * Create an instance of Feature meta. + * + * @param name Feature name. + * @param featureId Feature id. + * @param isCategoricalFeature Is categorical feature. + */ + public FeatureMeta(String name, int featureId, boolean isCategoricalFeature) { + this.name = name; + this.featureId = featureId; + this.isCategoricalFeature = isCategoricalFeature; + } + + /** */ + public int getFeatureId() { + return featureId; + } + + /** */ + public boolean isCategoricalFeature() { + return isCategoricalFeature; + } + + /** */ + public String getName() { + return name; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/Histogram.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/Histogram.java new file mode 100644 index 0000000000000..6784af1c57675 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/Histogram.java @@ -0,0 +1,64 @@ +/* + * 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.ml.dataset.feature; + +import java.io.Serializable; +import java.util.Optional; +import java.util.Set; + +/** + * Interface of histogram over type T. + * + * @param Type of object for histogram. + * @param Type of histogram that can be used in math operations with this histogram. + */ +public interface Histogram> extends Serializable { + /** + * Add object to histogram. + * + * @param val Value. + */ + public void addElement(T val); + + /** + * + * @return bucket ids. + */ + public Set buckets(); + + /** + * + * @param bucketId Bucket id. + * @return value in according to bucket id. + */ + public Optional getValue(Integer bucketId); + + /** + * @param other Other histogram. + * @return sum of this and other histogram. + */ + public H plus(H other); + + /** + * Compares histogram with other and returns true if they are equals + * + * @param other Other histogram. + * @return true if histograms are equal. + */ + public boolean isEqualTo(H other); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/ObjectHistogram.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/ObjectHistogram.java new file mode 100644 index 0000000000000..d894c3f7709ec --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/ObjectHistogram.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.ml.dataset.feature; + +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.TreeMap; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Basic implementation of {@link Histogram} that implements also {@link DistributionComputer}. + * + * @param Type of object for histogram. + */ +public class ObjectHistogram implements Histogram>, DistributionComputer { + /** Serial version uid. */ + private static final long serialVersionUID = -2708731174031404487L; + + /** Bucket mapping. */ + private final IgniteFunction bucketMapping; + + /** Mapping to counter. */ + private final IgniteFunction mappingToCntr; + + /** Histogram. */ + private final Map hist; + + /** + * Create an instance of ObjectHistogram. + * + * @param bucketMapping Bucket mapping. + * @param mappingToCntr Mapping to counter. + */ + public ObjectHistogram(IgniteFunction bucketMapping, + IgniteFunction mappingToCntr) { + + this.bucketMapping = bucketMapping; + this.mappingToCntr = mappingToCntr; + this.hist = new TreeMap<>(Integer::compareTo); + } + + /** {@inheritDoc} */ + @Override public void addElement(T val) { + Integer bucket = bucketMapping.apply(val); + Double cntrVal = mappingToCntr.apply(val); + + assert cntrVal >= 0; + Double bucketVal = hist.getOrDefault(bucket, 0.0); + hist.put(bucket, bucketVal + cntrVal); + } + + /** {@inheritDoc} */ + @Override public Set buckets() { + return hist.keySet(); + } + + /** {@inheritDoc} */ + @Override public Optional getValue(Integer bucketId) { + return Optional.ofNullable(hist.get(bucketId)); + } + + /** {@inheritDoc} */ + @Override public TreeMap computeDistributionFunction() { + TreeMap res = new TreeMap<>(); + + double accum = 0.0; + for (Integer bucket : hist.keySet()) { + accum += hist.get(bucket); + res.put(bucket, accum); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public ObjectHistogram plus(ObjectHistogram other) { + ObjectHistogram res = new ObjectHistogram<>(bucketMapping, mappingToCntr); + addTo(this.hist, res.hist); + addTo(other.hist, res.hist); + return res; + } + + /** + * Adds bucket values to target histogram. + * + * @param from From. + * @param to To. + */ + private void addTo(Map from, Map to) { + from.forEach((bucket, value) -> { + Double putVal = to.getOrDefault(bucket, 0.0); + to.put(bucket, putVal + value); + }); + } + + /** {@inheritDoc} */ + public boolean isEqualTo(ObjectHistogram other) { + Set totalBuckets = new HashSet<>(buckets()); + totalBuckets.addAll(other.buckets()); + if(totalBuckets.size() != buckets().size()) + return false; + + for(Integer bucketId : totalBuckets) { + double leftVal = hist.get(bucketId); + double rightVal = other.hist.get(bucketId); + if(Math.abs(leftVal - rightVal) > 0.001) + return false; + } + + return true; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/package-info.java new file mode 100644 index 0000000000000..a0c346b686b5f --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/feature/package-info.java @@ -0,0 +1,24 @@ +/* + + * 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 for helper classes over features such as {@link org.apache.ignite.ml.dataset.feature.ObjectHistogram} or + * {@link org.apache.ignite.ml.dataset.feature.FeatureMeta}. + */ +package org.apache.ignite.ml.dataset.feature; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetBuilder.java new file mode 100644 index 0000000000000..8707e3a78771c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetBuilder.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.ml.dataset.impl.bootstrapping; + +import java.util.Arrays; +import java.util.Iterator; +import org.apache.commons.math3.distribution.PoissonDistribution; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.UpstreamEntry; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Builder for bootstrapped dataset. Bootstrapped dataset consist of several subsamples created in according to random + * sampling with replacements selection of vectors from original dataset. This realization uses + * {@link BootstrappedVector} containing each vector from original sample with counters of repetitions + * for each subsample. As heuristic this implementation uses Poisson Distribution for generating counter values. + * + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + */ +public class BootstrappedDatasetBuilder implements PartitionDataBuilder { + /** Serial version uid. */ + private static final long serialVersionUID = 8146220902914010559L; + + /** Feature extractor. */ + private final IgniteBiFunction featureExtractor; + + /** Label extractor. */ + private final IgniteBiFunction lbExtractor; + + /** Samples count. */ + private final int samplesCnt; + + /** Subsample size. */ + private final double subsampleSize; + + /** + * Creates an instance of BootstrappedDatasetBuilder. + * + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param samplesCnt Samples count. + * @param subsampleSize Subsample size. + */ + public BootstrappedDatasetBuilder(IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, int samplesCnt, double subsampleSize) { + + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + this.samplesCnt = samplesCnt; + this.subsampleSize = subsampleSize; + } + + /** {@inheritDoc} */ + @Override public BootstrappedDatasetPartition build(Iterator> upstreamData, long upstreamDataSize, + EmptyContext ctx) { + + BootstrappedVector[] dataset = new BootstrappedVector[Math.toIntExact(upstreamDataSize)]; + + int cntr = 0; + PoissonDistribution poissonDistribution = new PoissonDistribution(subsampleSize); + while(upstreamData.hasNext()) { + UpstreamEntry nextRow = upstreamData.next(); + Vector features = featureExtractor.apply(nextRow.getKey(), nextRow.getValue()); + Double lb = lbExtractor.apply(nextRow.getKey(), nextRow.getValue()); + int[] repetitionCounters = new int[samplesCnt]; + Arrays.setAll(repetitionCounters, i -> poissonDistribution.sample()); + dataset[cntr++] = new BootstrappedVector(features, lb, repetitionCounters); + } + + return new BootstrappedDatasetPartition(dataset); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetPartition.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetPartition.java new file mode 100644 index 0000000000000..2155d1a159970 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedDatasetPartition.java @@ -0,0 +1,68 @@ +/* + * 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.ml.dataset.impl.bootstrapping; + +import java.util.Arrays; +import java.util.Iterator; +import org.jetbrains.annotations.NotNull; + +/** + * Partition of bootstrapped vectors. + */ +public class BootstrappedDatasetPartition implements AutoCloseable, Iterable { + /** Vectors. */ + private final BootstrappedVector[] vectors; + + /** + * Creates an instance of BootstrappedDatasetPartition. + * + * @param vectors Vectors. + */ + public BootstrappedDatasetPartition(BootstrappedVector[] vectors) { + this.vectors = vectors; + } + + /** + * Returns vector from dataset in according to row id. + * + * @param rowId Row id. + * @return Vector. + */ + public BootstrappedVector getRow(int rowId) { + return vectors[rowId]; + } + + /** + * Returns rows count. + * + * @return rows count. + */ + public int getRowsCount() { + return vectors.length; + } + + /** {@inheritDoc} */ + @NotNull @Override public Iterator iterator() { + return Arrays.stream(vectors).iterator(); + } + + /** {@inheritDoc} */ + @Override public void close() throws Exception { + //NOP + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedVector.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedVector.java new file mode 100644 index 0000000000000..aedd0fd80b8b3 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/BootstrappedVector.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.dataset.impl.bootstrapping; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Arrays; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.structures.LabeledVector; + +/** + * Represents vector with repetitions counters for subsamples in bootstrapped dataset. + * Each counter shows the number of repetitions of the vector for the n-th sample. + */ +public class BootstrappedVector extends LabeledVector { + /** Serial version uid. */ + private static final long serialVersionUID = -4583008673032917259L; + + /** Counters show the number of repetitions of the vector for the n-th sample. */ + private int[] counters; + + /** + * Creates an instance of BootstrappedVector. + * + * @param features Features. + * @param lb Label. + * @param counters Repetitions counters. + */ + public BootstrappedVector(Vector features, double lb, int[] counters) { + super(features, lb); + this.counters = counters; + } + + /** + * @return repetitions counters vector. + */ + public int[] counters() { + return counters; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) + return false; + BootstrappedVector vector = (BootstrappedVector)o; + return Arrays.equals(counters, vector.counters); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int result = super.hashCode(); + result = 31 * result + Arrays.hashCode(counters); + return result; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + super.writeExternal(out); + out.writeObject(counters); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + super.readExternal(in); + counters = (int[]) in.readObject(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/package-info.java new file mode 100644 index 0000000000000..5fcf629b373f7 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/bootstrapping/package-info.java @@ -0,0 +1,23 @@ +/* + + * 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. + */ + +/** + * + * Base package for bootstrapped implementation of machine learning dataset. + */ +package org.apache.ignite.ml.dataset.impl.bootstrapping; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java index bbbb2a9a2bfcd..559dffff1c9e8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java @@ -17,72 +17,93 @@ package org.apache.ignite.ml.tree.randomforest; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.composition.predictionsaggregator.OnMajorityPredictionsAggregator; -import org.apache.ignite.ml.composition.predictionsaggregator.PredictionsAggregator; -import org.apache.ignite.ml.trainers.DatasetTrainer; -import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer; -import org.apache.ignite.ml.tree.DecisionTreeNode; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.dataset.feature.ObjectHistogram; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedDatasetPartition; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.randomforest.data.TreeRoot; +import org.apache.ignite.ml.tree.randomforest.data.impurity.GiniHistogram; +import org.apache.ignite.ml.tree.randomforest.data.impurity.GiniHistogramsComputer; +import org.apache.ignite.ml.tree.randomforest.data.impurity.ImpurityHistogramsComputer; +import org.apache.ignite.ml.tree.randomforest.data.statistics.ClassifierLeafValuesComputer; +import org.apache.ignite.ml.tree.randomforest.data.statistics.LeafValuesComputer; /** - * Random forest classifier trainer. + * Classifier trainer based on RandomForest algorithm. */ -public class RandomForestClassifierTrainer extends RandomForestTrainer { +public class RandomForestClassifierTrainer + extends RandomForestTrainer, GiniHistogram, RandomForestClassifierTrainer> { + /** Label mapping. */ + private Map lblMapping = new HashMap<>(); + /** - * Constructs new instance of RandomForestClassifierTrainer. + * Constructs an instance of RandomForestClassifierTrainer. * - * @param predictionsAggregator Predictions aggregator. - * @param featureVectorSize Feature vector size. - * @param maximumFeaturesCntPerMdl Number of features to draw from original features vector to train each model. - * @param ensembleSize Ensemble size. - * @param samplePartSizePerMdl Size of sample part in percent to train one model. - * @param maxDeep Max decision tree deep. - * @param minImpurityDecrease Min impurity decrease. + * @param meta Features meta. */ - public RandomForestClassifierTrainer(PredictionsAggregator predictionsAggregator, - int featureVectorSize, - int maximumFeaturesCntPerMdl, - int ensembleSize, - double samplePartSizePerMdl, - int maxDeep, - double minImpurityDecrease) { + public RandomForestClassifierTrainer(List meta) { + super(meta); + } - super(predictionsAggregator, featureVectorSize, maximumFeaturesCntPerMdl, - ensembleSize, samplePartSizePerMdl, maxDeep, minImpurityDecrease); + /** {@inheritDoc} */ + @Override protected RandomForestClassifierTrainer instance() { + return this; } /** - * Constructs new instance of RandomForestClassifierTrainer. + * Aggregates all unique labels from dataset and assigns integer id value for each label. + * This id can be used as index in arrays or lists. * - * @param featureVectorSize Feature vector size. - * @param maximumFeaturesCntPerMdl Number of features to draw from original features vector to train each model. - * @param ensembleSize Ensemble size. - * @param samplePartSizePerMdl Size of sample part in percent to train one model. - * @param maxDeep Max decision tree deep. - * @param minImpurityDecrease Min impurity decrease. + * @param dataset Dataset. */ - public RandomForestClassifierTrainer(int featureVectorSize, - int maximumFeaturesCntPerMdl, - int ensembleSize, - double samplePartSizePerMdl, - int maxDeep, double minImpurityDecrease) { + @Override protected void init(Dataset dataset) { + Set uniqLabels = dataset.compute( + x -> { + Set labels = new HashSet<>(); + for (int i = 0; i < x.getRowsCount(); i++) + labels.add(x.getRow(i).label()); + return labels; + }, + (l, r) -> { + if (l == null) + return r; + if (r == null) + return l; + Set lbls = new HashSet<>(); + lbls.addAll(l); + lbls.addAll(r); + return lbls; + } + ); - this(new OnMajorityPredictionsAggregator(), featureVectorSize, maximumFeaturesCntPerMdl, - ensembleSize, samplePartSizePerMdl, maxDeep, minImpurityDecrease); + int i = 0; + for (Double label : uniqLabels) + lblMapping.put(label, i++); + + super.init(dataset); } /** {@inheritDoc} */ - @Override protected DatasetTrainer buildDatasetTrainerForModel() { - return new DecisionTreeClassificationTrainer(maxDeep, minImpurityDecrease).withUseIndex(useIndex); + @Override protected ModelsComposition buildComposition(List models) { + return new ModelsComposition(models, new OnMajorityPredictionsAggregator()); } - /** - * Sets useIndex parameter and returns trainer instance. - * - * @param useIndex Use index. - * @return Decision tree trainer. - */ - public RandomForestClassifierTrainer withUseIndex(boolean useIndex) { - this.useIndex = useIndex; - return this; + /** {@inheritDoc} */ + @Override protected ImpurityHistogramsComputer createImpurityHistogramsComputer() { + return new GiniHistogramsComputer(lblMapping); + } + + /** {@inheritDoc} */ + @Override protected LeafValuesComputer> createLeafStatisticsAggregator() { + return new ClassifierLeafValuesComputer(lblMapping); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainer.java index 009fff28fbc18..ab1d0361ee4c7 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainer.java @@ -17,73 +17,49 @@ package org.apache.ignite.ml.tree.randomforest; +import java.util.List; +import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.composition.predictionsaggregator.MeanValuePredictionsAggregator; -import org.apache.ignite.ml.composition.predictionsaggregator.PredictionsAggregator; -import org.apache.ignite.ml.trainers.DatasetTrainer; -import org.apache.ignite.ml.tree.DecisionTreeNode; -import org.apache.ignite.ml.tree.DecisionTreeRegressionTrainer; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.tree.randomforest.data.TreeRoot; +import org.apache.ignite.ml.tree.randomforest.data.impurity.ImpurityHistogramsComputer; +import org.apache.ignite.ml.tree.randomforest.data.impurity.MSEHistogram; +import org.apache.ignite.ml.tree.randomforest.data.impurity.MSEHistogramComputer; +import org.apache.ignite.ml.tree.randomforest.data.statistics.LeafValuesComputer; +import org.apache.ignite.ml.tree.randomforest.data.statistics.MeanValueStatistic; +import org.apache.ignite.ml.tree.randomforest.data.statistics.RegressionLeafValuesComputer; /** - * Random forest regression trainer. + * Regression trainer based on RandomForest algorithm. */ -public class RandomForestRegressionTrainer extends RandomForestTrainer { +public class RandomForestRegressionTrainer + extends RandomForestTrainer { /** - * Constructs new instance of RandomForestRegressionTrainer. + * Constructs an instance of RandomForestRegressionTrainer. * - * @param predictionsAggregator Predictions aggregator. - * @param featureVectorSize Feature vector size. - * @param maximumFeaturesCntPerMdl Number of features to draw from original features vector to train each model. - * @param ensembleSize Ensemble size. - * @param samplePartSizePerMdl Size of sample part in percent to train one model. - * @param maxDeep Max decision tree deep. - * @param minImpurityDecrease Min impurity decrease. + * @param meta Features meta. */ - public RandomForestRegressionTrainer(PredictionsAggregator predictionsAggregator, - int featureVectorSize, - int maximumFeaturesCntPerMdl, - int ensembleSize, - double samplePartSizePerMdl, - int maxDeep, - double minImpurityDecrease) { - - super(predictionsAggregator, featureVectorSize, maximumFeaturesCntPerMdl, - ensembleSize, samplePartSizePerMdl, maxDeep, minImpurityDecrease); + public RandomForestRegressionTrainer(List meta) { + super(meta); } - /** - * Constructs new instance of RandomForestRegressionTrainer. - * - * @param featureVectorSize Feature vector size. - * @param maximumFeaturesCntPerMdl Number of features to draw from original features vector to train each model. - * @param ensembleSize Ensemble size. - * @param samplePartSizePerMdl Size of sample part in percent to train one model. - * @param maxDeep Max decision tree deep. - * @param minImpurityDecrease Min impurity decrease. - */ - public RandomForestRegressionTrainer(int featureVectorSize, - int maximumFeaturesCntPerMdl, - int ensembleSize, - double samplePartSizePerMdl, - int maxDeep, - double minImpurityDecrease) { + /** {@inheritDoc} */ + @Override protected RandomForestRegressionTrainer instance() { + return this; + } - this(new MeanValuePredictionsAggregator(), featureVectorSize, maximumFeaturesCntPerMdl, - ensembleSize, samplePartSizePerMdl, maxDeep, minImpurityDecrease); + /** {@inheritDoc} */ + @Override protected ModelsComposition buildComposition(List models) { + return new ModelsComposition(models, new MeanValuePredictionsAggregator()); } /** {@inheritDoc} */ - @Override protected DatasetTrainer buildDatasetTrainerForModel() { - return new DecisionTreeRegressionTrainer(maxDeep, minImpurityDecrease).withUseIndex(useIndex); + @Override protected ImpurityHistogramsComputer createImpurityHistogramsComputer() { + return new MSEHistogramComputer(); } - /** - * Sets useIndex parameter and returns trainer instance. - * - * @param useIndex Use index. - * @return Decision tree trainer. - */ - public RandomForestRegressionTrainer withUseIndex(boolean useIndex) { - this.useIndex = useIndex; - return this; + /** {@inheritDoc} */ + @Override protected LeafValuesComputer createLeafStatisticsAggregator() { + return new RegressionLeafValuesComputer(); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java index 8608f099e4bb0..cb25aa32df1ad 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java @@ -17,45 +17,359 @@ package org.apache.ignite.ml.tree.randomforest; -import org.apache.ignite.ml.composition.BaggingModelTrainer; -import org.apache.ignite.ml.composition.predictionsaggregator.PredictionsAggregator; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.Random; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.feature.BucketMeta; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedDatasetBuilder; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedDatasetPartition; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.trainers.DatasetTrainer; +import org.apache.ignite.ml.tree.randomforest.data.FeaturesCountSelectionStrategies; +import org.apache.ignite.ml.tree.randomforest.data.NodeId; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; +import org.apache.ignite.ml.tree.randomforest.data.TreeNode; +import org.apache.ignite.ml.tree.randomforest.data.TreeRoot; +import org.apache.ignite.ml.tree.randomforest.data.impurity.ImpurityComputer; +import org.apache.ignite.ml.tree.randomforest.data.impurity.ImpurityHistogramsComputer; +import org.apache.ignite.ml.tree.randomforest.data.statistics.LeafValuesComputer; +import org.apache.ignite.ml.tree.randomforest.data.statistics.NormalDistributionStatistics; +import org.apache.ignite.ml.tree.randomforest.data.statistics.NormalDistributionStatisticsComputer; /** - * Abstract random forest trainer. + * Class represents a realization of Random Forest algorithm. Main idea of this realization is that at each learning + * iteration it tries to aggregate statistics on impurity for each corner nodes (leaves that may be splitted on a new + * iteration) in each trees in random forest. It requires one map-reduce operation over learning dataset. After such + * aggregation the algorithm selects split points for each corner nodes of create leaf nodes. The algorithm stops when + * there is no splitting for nodes in trees. At last stage the algorithm aggregates statistics on labels for leaf nodes + * in one map-reduce step and sets values to leafs based these statistics. + * + * @param Type of statistics aggregator for leaf values computing. + * @param Type of impurity computer specific for algorithm. + * @param Type of child of RandomForestTrainer using in with-methods. */ -public abstract class RandomForestTrainer extends BaggingModelTrainer { - /** Max decision tree deep. */ - protected final int maxDeep; +public abstract class RandomForestTrainer, + T extends RandomForestTrainer> extends DatasetTrainer { + /** Bucket size factor. */ + private static final double BUCKET_SIZE_FACTOR = (1 / 10.0); + + /** Count of trees. */ + private int cntOfTrees = 1; + + /** Subsample size. */ + private double subsampleSize = 1.0; + + /** Max depth. */ + private int maxDepth = 5; + + /** Min impurity delta. */ + private double minImpurityDelta = 0.0; + + /** Features Meta. */ + private List meta; + + /** Features per tree. */ + private int featuresPerTree; + + /** Seed. */ + private long seed = System.currentTimeMillis(); + + /** Random generator. */ + private Random random = new Random(seed); + + /** Nodes to learn selection strategy. */ + private Function, List> nodesToLearnSelectionStrgy = this::defaultNodesToLearnSelectionStrgy; + + /** + * Create an instance of RandomForestTrainer. + * + * @param meta Features Meta. + */ + public RandomForestTrainer(List meta) { + this.meta = meta; + this.featuresPerTree = FeaturesCountSelectionStrategies.ALL.apply(meta); + } + + /** {@inheritDoc} */ + @Override public ModelsComposition fit(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + List models = null; + try (Dataset dataset = datasetBuilder.build( + new EmptyContextBuilder<>(), + new BootstrappedDatasetBuilder<>(featureExtractor, lbExtractor, cntOfTrees, subsampleSize))) { + + init(dataset); + models = fit(dataset); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + assert models != null; + return buildComposition(models); + } + + /** + * @return an instance of current object with valid type in according to inheritance. + */ + protected abstract T instance(); + + /** + * @param cntOfTrees Count of trees. + * @return an instance of current object with valid type in according to inheritance. + */ + public T withCountOfTrees(int cntOfTrees) { + this.cntOfTrees = cntOfTrees; + return instance(); + } + + /** + * @param subsampleSize Subsample size. + * @return an instance of current object with valid type in according to inheritance. + */ + public T withSubsampleSize(double subsampleSize) { + this.subsampleSize = subsampleSize; + return instance(); + } + + /** + * @param maxDepth Max depth. + * @return an instance of current object with valid type in according to inheritance. + */ + public T withMaxDepth(int maxDepth) { + this.maxDepth = maxDepth; + return instance(); + } - /** Min impurity decrease. */ - protected final double minImpurityDecrease; + /** + * @param minImpurityDelta Min impurity delta. + * @return an instance of current object with valid type in according to inheritance. + */ + public T withMinImpurityDelta(double minImpurityDelta) { + this.minImpurityDelta = minImpurityDelta; + return instance(); + } - /** Use index structure instead of using sorting while decision tree learning. */ - protected boolean useIndex = false; + /** + * @param strgy Strgy. + * @return an instance of current object with valid type in according to inheritance. + */ + public T withFeaturesCountSelectionStrgy(Function, Integer> strgy) { + this.featuresPerTree = strgy.apply(meta); + return instance(); + } /** - * Constructs new instance of BaggingModelTrainer. + * Sets strategy for selection nodes from learning queue in each iteration. * - * @param predictionsAggregator Predictions aggregator. - * @param featureVectorSize Feature vector size. - * @param maximumFeaturesCntPerMdl Number of features to draw from original features vector to train each model. - * @param ensembleSize Ensemble size. - * @param samplePartSizePerMdl Size of sample part in percent to train one model. - * @param maxDeep Max decision tree deep. - * @param minImpurityDecrease Min impurity decrease. + * @param strgy Strgy. */ - public RandomForestTrainer(PredictionsAggregator predictionsAggregator, - int featureVectorSize, - int maximumFeaturesCntPerMdl, - int ensembleSize, - double samplePartSizePerMdl, - int maxDeep, - double minImpurityDecrease) { + public T withNodesToLearnSelectionStrgy(Function, List> strgy) { + this.nodesToLearnSelectionStrgy = strgy; + return instance(); + } - super(predictionsAggregator, featureVectorSize, maximumFeaturesCntPerMdl, - ensembleSize, samplePartSizePerMdl); + /** + * @param seed Seed. + * @return an instance of current object with valid type in according to inheritance. + */ + public T withSeed(long seed) { + this.seed = seed; + this.random = new Random(seed); + return instance(); + } - this.maxDeep = maxDeep; - this.minImpurityDecrease = minImpurityDecrease; + /** + * Init-step before learning. It may be useful collecting labels statistics step for classification. + * + * @param dataset Dataset. + */ + protected void init(Dataset dataset) { } + + /** + * Trains model based on the specified data. + * + * @param dataset Dataset. + * @return list of decision trees. + */ + private List fit(Dataset dataset) { + Queue treesQueue = createRootsQueue(); + ArrayList roots = initTrees(treesQueue); + Map histMeta = computeHistogramMeta(meta, dataset); + + ImpurityHistogramsComputer histogramsComputer = createImpurityHistogramsComputer(); + while (!treesQueue.isEmpty()) { + Map nodesToLearn = getNodesToLearn(treesQueue); + Map> nodesImpHists = histogramsComputer + .aggregateImpurityStatistics(roots, histMeta, nodesToLearn, dataset); + if (nodesToLearn.size() != nodesImpHists.size()) + throw new IllegalStateException(); + + for (NodeId nodeId : nodesImpHists.keySet()) + split(treesQueue, nodesToLearn, nodesImpHists.get(nodeId)); + } + + createLeafStatisticsAggregator().setValuesForLeaves(roots, dataset); + return roots; + } + + /** + * Split node with NodeId if need. + * + * @param learningQueue Learning queue. + * @param nodesToLearn Nodes to learn at current iteration. + * @param nodeImpurityHistograms Impurity histograms on current iteration. + */ + private void split(Queue learningQueue, Map nodesToLearn, + ImpurityHistogramsComputer.NodeImpurityHistograms nodeImpurityHistograms) { + + TreeNode cornerNode = nodesToLearn.get(nodeImpurityHistograms.getNodeId()); + Optional bestSplit = nodeImpurityHistograms.findBestSplit(); + + if (needSplit(cornerNode, bestSplit)) { + List children = bestSplit.get().split(cornerNode); + learningQueue.addAll(children); + } + else { + if (bestSplit.isPresent()) + bestSplit.get().createLeaf(cornerNode); + else { + cornerNode.setImpurity(Double.NEGATIVE_INFINITY); + cornerNode.toLeaf(0.0); + } + } + } + + /** + * Creates an instance of Histograms Computer corresponding to RF implementation. + */ + protected abstract ImpurityHistogramsComputer createImpurityHistogramsComputer(); + + /** + * Creates an instance of Leaf Statistics Aggregator corresponding to RF implementation. + */ + protected abstract LeafValuesComputer createLeafStatisticsAggregator(); + + /** + * Creates list of trees. + * + * @param treesQueue Trees queue. + * @return List of trees. + */ + protected ArrayList initTrees(Queue treesQueue) { + assert featuresPerTree > 0; + ArrayList roots = new ArrayList<>(); + + List allFeatureIds = IntStream.range(0, meta.size()).boxed().collect(Collectors.toList()); + for (TreeNode node : treesQueue) { + Collections.shuffle(allFeatureIds, random); + Set featuresSubspace = allFeatureIds.stream() + .limit(featuresPerTree).collect(Collectors.toSet()); + roots.add(new TreeRoot(node, featuresSubspace)); + } + + return roots; + } + + /** + * Compute bucket metas based on feature metas and learning dataset. + * + * @param meta Features meta. + * @param dataset Dataset. + * @return bucket metas. + */ + private Map computeHistogramMeta(List meta, + Dataset dataset) { + + List stats = new NormalDistributionStatisticsComputer() + .computeStatistics(meta, dataset); + + Map bucketsMeta = new HashMap<>(); + for (int i = 0; i < stats.size(); i++) { + BucketMeta bucketMeta = new BucketMeta(meta.get(i)); + if (!bucketMeta.getFeatureMeta().isCategoricalFeature()) { + NormalDistributionStatistics stat = stats.get(i); + bucketMeta.setMinVal(stat.min()); + bucketMeta.setBucketSize(stat.std() * BUCKET_SIZE_FACTOR); + } + bucketsMeta.put(i, bucketMeta); + } + return bucketsMeta; + } + + /** + * Creates an initial nodes queue for learning based on countOfTrees. Each of these nodes represents a root decision + * trees in random forest. + * + * @return initial nodes queue. + */ + private Queue createRootsQueue() { + Queue roots = new LinkedList<>(); + for (int i = 0; i < cntOfTrees; i++) + roots.add(new TreeNode(1, i)); + return roots; + } + + /** + * Select set of nodes for leaning from queue based in nodesToLearnSelectionStrategy. + * + * @param queue Learning queue. + * @return collection of nodes for learning iterations. + */ + private Map getNodesToLearn(Queue queue) { + return nodesToLearnSelectionStrgy.apply(queue).stream() + .collect(Collectors.toMap(TreeNode::getId, node -> node)); + } + + /** + * Default nodesToLearnSelectionStrategy that returns all nodes from queue. + * + * @param queue Queue. + * @return List of nodes to learn. + */ + private List defaultNodesToLearnSelectionStrgy(Queue queue) { + List res = new ArrayList<>(queue); + queue.clear(); + return res; + } + + /** + * Check current note for the need for splitting. + * + * @param parentNode Parent node. + * @param split Best node split. + * @return true if split is needed. + */ + boolean needSplit(TreeNode parentNode, Optional split) { + return split.isPresent() && parentNode.getImpurity() - split.get().getImpurity() > minImpurityDelta && + parentNode.getDepth() < (maxDepth + 1); + } + + /** + * Returns composition of built trees. + * + * @param models Models. + * @return composition of built trees. + */ + protected abstract ModelsComposition buildComposition(List models); + } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/FeaturesCountSelectionStrategies.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/FeaturesCountSelectionStrategies.java new file mode 100644 index 0000000000000..a24627747b4d8 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/FeaturesCountSelectionStrategies.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.ml.tree.randomforest.data; + +import java.util.List; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.math.functions.IgniteFunction; + +/** + * Class contains a default implementations of some features count selection strategies for random forest. + */ +public class FeaturesCountSelectionStrategies { + /** */ + public static IgniteFunction, Integer> SQRT = (List meta) -> { + return (int)Math.sqrt(meta.size()); + }; + + /** */ + public static IgniteFunction, Integer> ALL = (List meta) -> { + return meta.size(); + }; + + /** */ + public static IgniteFunction, Integer> LOG2 = (List meta) -> { + return (int)(Math.log(meta.size()) / Math.log(2)); + }; + + /** */ + public static IgniteFunction, Integer> ONE_THIRD = (List meta) -> { + return (int)(meta.size() / 3); + }; +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeId.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeId.java new file mode 100644 index 0000000000000..2f40af352630a --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeId.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.randomforest.data; + +import org.apache.ignite.lang.IgniteBiTuple; + +/** + * Class represents Node id in Random Forest consisting of tree id and node id in tree in according to + * breadth-first search in tree. + */ +public class NodeId extends IgniteBiTuple { + /** Serial version uid. */ + private static final long serialVersionUID = 4400852013136423333L; + + /** + * Create an instance of NodeId. + * + * @param treeId Tree id. + * @param nodeId Node id. + */ + public NodeId(Integer treeId, Long nodeId) { + super(treeId, nodeId); + } + + /** + * + * @return Tree id. + */ + public int treeId() { + return get1(); + } + + /** + * + * @return Node id. + */ + public long nodeId() { + return get2(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeSplit.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeSplit.java new file mode 100644 index 0000000000000..52d0b745455c1 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/NodeSplit.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.ml.tree.randomforest.data; + +import java.util.List; + +/** + * Class represents a split point for decision tree. + */ +public class NodeSplit { + /** Feature id in feature vector. */ + private final int featureId; + + /** Feature split value. */ + private final double value; + + /** Impurity at this split point. */ + private final double impurity; + + /** + * Creates an instance of NodeSplit. + * + * @param featureId Feature id. + * @param value Feature split value. + * @param impurity Impurity value. + */ + public NodeSplit(int featureId, double value, double impurity) { + this.featureId = featureId; + this.value = value; + this.impurity = impurity; + } + + /** + * Split node from parameter onto two children nodes. + * + * @param node Node. + * @return list of children. + */ + public List split(TreeNode node) { + List children = node.toConditional(featureId, value); + node.setImpurity(impurity); + return children; + } + + /** + * Convert node to leaf. + * + * @param node Node. + */ + public void createLeaf(TreeNode node) { + node.setImpurity(impurity); + node.toLeaf(0.0); //values will be set in last stage if training + } + + /** */ + public double getImpurity() { + return impurity; + } + + /** */ + public double getValue() { + return value; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeNode.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeNode.java new file mode 100644 index 0000000000000..eb06143b6897b --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeNode.java @@ -0,0 +1,200 @@ +/* + * 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.ml.tree.randomforest.data; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import org.apache.ignite.ml.Model; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Decision tree node class. + */ +public class TreeNode implements Model, Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = -8546263332508653661L; + + /** + * Type of node. + */ + public enum Type { + /** Unknown. */ + UNKNOWN, + + /** Leaf node. */ + LEAF, + + /** Conditional node. */ + CONDITIONAL + } + + /** Id. */ + private final NodeId id; + + /** Feature id. */ + private int featureId; + + /** Value. */ + private double value; + + /** Type. */ + private Type type; + + /** Impurity. */ + private double impurity; + + /** Depth. */ + private int depth; + + /** Left branch. */ + private TreeNode left; + + /** Right branch. */ + private TreeNode right; + + /** + * Create an instance of TreeNode. + * + * @param id Id in according to breadth-first search ordering. + * @param treeId Tree id. + */ + public TreeNode(long id, int treeId) { + this.id = new NodeId(treeId, id); + this.value = -1; + this.type = Type.UNKNOWN; + this.impurity = Double.POSITIVE_INFINITY; + this.depth = 1; + } + + /** {@inheritDoc} */ + public Double apply(Vector features) { + assert type != Type.UNKNOWN; + + if (type == Type.LEAF) + return value; + else { + if (features.get(featureId) <= value) + return left.apply(features); + else + return right.apply(features); + } + } + + /** + * Returns leaf node for feature vector in according to decision tree. + * + * @param features Features. + * @return Node. + */ + public NodeId predictNextNodeKey(Vector features) { + switch (type) { + case UNKNOWN: + return id; + case LEAF: + return id; + default: + if (features.get(featureId) <= value) + return left.predictNextNodeKey(features); + else + return right.predictNextNodeKey(features); + } + } + + /** + * Convert node to conditional node. + * + * @param featureId Feature id. + * @param value Value. + */ + public List toConditional(int featureId, double value) { + assert type == Type.UNKNOWN; + + toLeaf(value); + left = new TreeNode(2 * id.nodeId(), id.treeId()); + right = new TreeNode(2 * id.nodeId() + 1, id.treeId()); + this.type = Type.CONDITIONAL; + this.featureId = featureId; + + left.depth = right.depth = depth + 1; + return Arrays.asList(left, right); + } + + /** + * Convert node to leaf. + * + * @param value Value. + */ + public void toLeaf(double value) { + assert type == Type.UNKNOWN; + + this.value = value; + this.type = Type.LEAF; + + this.left = null; + this.right = null; + } + + /** */ + public NodeId getId() { + return id; + } + + /** */ + public void setValue(double value) { + this.value = value; + } + + /** */ + public Type getType() { + return type; + } + + /** */ + public void setImpurity(double impurity) { + this.impurity = impurity; + } + + /** + * @return impurity in current node. + */ + public double getImpurity() { + return impurity; + } + + /** + * @return depth of current node. + */ + public int getDepth() { + return depth; + } + + /** + * @return right subtree. + */ + public TreeNode getLeft() { + return left; + } + + /** + * @return left subtree. + */ + public TreeNode getRight() { + return right; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeRoot.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeRoot.java new file mode 100644 index 0000000000000..e47868d14c787 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/TreeRoot.java @@ -0,0 +1,86 @@ +/* + * 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.ml.tree.randomforest.data; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import org.apache.ignite.ml.Model; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Tree root class. + */ +public class TreeRoot implements Model { + /** Serial version uid. */ + private static final long serialVersionUID = 531797299171329057L; + + /** Root node. */ + private TreeNode node; + + /** Used features. */ + private Set usedFeatures; + + /** + * Create an instance of TreeRoot. + * + * @param root Root. + * @param usedFeatures Used features. + */ + public TreeRoot(TreeNode root, Set usedFeatures) { + this.node = root; + this.usedFeatures = usedFeatures; + } + + /** {@inheritDoc} */ + @Override public Double apply(Vector vector) { + return node.apply(vector); + } + + /** */ + public Set getUsedFeatures() { + return usedFeatures; + } + + /** */ + public TreeNode getRootNode() { + return node; + } + + /** + * @return all leafs in tree. + */ + public List getLeafs() { + List res = new ArrayList<>(); + getLeafs(node, res); + return res; + } + + /** + * @param root Root. + * @param res Result list. + */ + private void getLeafs(TreeNode root, List res) { + if (root.getType() == TreeNode.Type.LEAF) + res.add(root); + else { + getLeafs(root.getLeft(), res); + getLeafs(root.getRight(), res); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogram.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogram.java new file mode 100644 index 0000000000000..80a07a7e8e27c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogram.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.ml.tree.randomforest.data.impurity; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.stream.Collectors; +import org.apache.ignite.ml.dataset.feature.BucketMeta; +import org.apache.ignite.ml.dataset.feature.ObjectHistogram; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; + +/** + * Class contains implementation of splitting point finding algorithm based on Gini metric (see + * https://en.wikipedia.org/wiki/Gini_coefficient) and represents a set of histograms in according to this metric. + */ +public class GiniHistogram extends ImpurityHistogram implements ImpurityComputer { + /** Serial version uid. */ + private static final long serialVersionUID = 5780670356098827667L; + + /** Bucket meta. */ + private final BucketMeta bucketMeta; + + /** Sample id. */ + private final int sampleId; + + /** Hists of counters for each labels. */ + private final ArrayList> hists; + + /** Label mapping to internal representation. */ + private final Map lblMapping; + + /** Bucket ids. */ + private final Set bucketIds; + + /** + * Creates an instance of GiniHistogram. + * + * @param sampleId Sample id. + * @param lblMapping Label mapping. + * @param bucketMeta Bucket meta. + */ + public GiniHistogram(int sampleId, Map lblMapping, BucketMeta bucketMeta) { + super(bucketMeta.getFeatureMeta().getFeatureId()); + this.hists = new ArrayList<>(lblMapping.size()); + this.sampleId = sampleId; + this.bucketMeta = bucketMeta; + this.lblMapping = lblMapping; + + for (int i = 0; i < lblMapping.size(); i++) + hists.add(new ObjectHistogram<>(this::bucketMap, this::counterMap)); + + this.bucketIds = new TreeSet<>(); + } + + /** {@inheritDoc} */ + @Override public void addElement(BootstrappedVector vector) { + Integer lblId = lblMapping.get(vector.label()); + hists.get(lblId).addElement(vector); + } + + /** {@inheritDoc} */ + @Override public Optional getValue(Integer bucketId) { + throw new NotImplementedException(); + } + + /** {@inheritDoc} */ + @Override public GiniHistogram plus(GiniHistogram other) { + GiniHistogram res = new GiniHistogram(sampleId, lblMapping, bucketMeta); + res.bucketIds.addAll(this.bucketIds); + res.bucketIds.addAll(other.bucketIds); + for (int i = 0; i < hists.size(); i++) + res.hists.set(i, this.hists.get(i).plus(other.hists.get(i))); + return res; + } + + /** {@inheritDoc} */ + @Override public Optional findBestSplit() { + if (bucketIds.size() < 2) + return Optional.empty(); + + double bestImpurity = Double.POSITIVE_INFINITY; + double bestSplitVal = Double.NEGATIVE_INFINITY; + int bestBucketId = -1; + + List> countersDistribPerCls = hists.stream() + .map(ObjectHistogram::computeDistributionFunction) + .collect(Collectors.toList()); + + double[] totalSampleCntPerLb = countersDistribPerCls.stream() + .mapToDouble(x -> x.isEmpty() ? 0.0 : x.lastEntry().getValue()) + .toArray(); + + Map lastLeftValues = new HashMap<>(); + for (int i = 0; i < lblMapping.size(); i++) + lastLeftValues.put(i, 0.0); + + for (Integer bucketId : bucketIds) { + double totalToleftCnt = 0; + double totalToRightCnt = 0; + + double leftImpurity = 0; + double rightImpurity = 0; + + //Compute number of samples left and right in according to split by bucketId + for (int lbId = 0; lbId < lblMapping.size(); lbId++) { + Double left = countersDistribPerCls.get(lbId).get(bucketId); + if (left == null) + left = lastLeftValues.get(lbId); + + totalToleftCnt += left; + totalToRightCnt += totalSampleCntPerLb[lbId] - left; + + lastLeftValues.put(lbId, left); + } + + for (int lbId = 0; lbId < lblMapping.size(); lbId++) { + //count of samples with label [corresponding lblId] to the left of bucket + Double toLeftCnt = countersDistribPerCls.get(lbId).getOrDefault(bucketId, lastLeftValues.get(lbId)); + + if (toLeftCnt > 0) + leftImpurity += Math.pow(toLeftCnt, 2) / totalToleftCnt; + + //number of samples to the right of bucket = total samples count - toLeftCnt + double toRightCnt = totalSampleCntPerLb[lbId] - toLeftCnt; + if (toRightCnt > 0) + rightImpurity += (Math.pow(toRightCnt, 2)) / totalToRightCnt; + } + + double impurityInBucket = -(leftImpurity + rightImpurity); + if (impurityInBucket <= bestImpurity) { + bestImpurity = impurityInBucket; + bestSplitVal = bucketMeta.bucketIdToValue(bucketId); + bestBucketId = bucketId; + } + } + + return checkAndReturnSplitValue(bestBucketId, bestSplitVal, bestImpurity); + } + + /** {@inheritDoc} */ + @Override public Set buckets() { + return bucketIds; + } + + /** + * Returns counters histogram for class-label. + * + * @param lbl Label. + * @return counters histogram for class-label. + */ + ObjectHistogram getHistForLabel(Double lbl) { + return hists.get(lblMapping.get(lbl)); + } + + /** + * Maps vector to counter value. + * + * @param vec Vector. + * @return Counter value. + */ + private Double counterMap(BootstrappedVector vec) { + return (double)vec.counters()[sampleId]; + } + + /** + * Maps vector to bucket id. + * + * @param vec Vector. + * @return Bucket id. + */ + private Integer bucketMap(BootstrappedVector vec) { + int bucketId = bucketMeta.getBucketId(vec.features().get(featureId)); + this.bucketIds.add(bucketId); + return bucketId; + } + + /** {@inheritDoc} */ + @Override public boolean isEqualTo(GiniHistogram other) { + HashSet unionBuckets = new HashSet<>(buckets()); + unionBuckets.addAll(other.bucketIds); + if (unionBuckets.size() != bucketIds.size()) + return false; + + HashSet unionMappings = new HashSet<>(lblMapping.keySet()); + unionMappings.addAll(other.lblMapping.keySet()); + if (unionMappings.size() != lblMapping.size()) + return false; + + for (Double lbl : unionMappings) { + if (lblMapping.get(lbl) != other.lblMapping.get(lbl)) + return false; + + ObjectHistogram thisHist = getHistForLabel(lbl); + ObjectHistogram otherHist = other.getHistForLabel(lbl); + if (!thisHist.isEqualTo(otherHist)) + return false; + } + + return true; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogramsComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogramsComputer.java new file mode 100644 index 0000000000000..8c9dc95d715fb --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniHistogramsComputer.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.ml.tree.randomforest.data.impurity; + +import java.util.Map; +import org.apache.ignite.ml.dataset.feature.BucketMeta; + +/** + * Implementation of {@link ImpurityHistogramsComputer} for classification task. + */ +public class GiniHistogramsComputer extends ImpurityHistogramsComputer { + /** Serial version uid. */ + private static final long serialVersionUID = 3672921182944932748L; + + /** Label mapping. */ + private final Map lblMapping; + + /** + * Creates an instance of GiniHistogramsComputer. + * + * @param lblMapping Lbl mapping. + */ + public GiniHistogramsComputer(Map lblMapping) { + this.lblMapping = lblMapping; + } + + /** {@inheritDoc} */ + @Override protected GiniHistogram createImpurityComputerForFeature(int sampleId, BucketMeta meta) { + return new GiniHistogram(sampleId, lblMapping, meta); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityComputer.java new file mode 100644 index 0000000000000..0684415c87337 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityComputer.java @@ -0,0 +1,38 @@ +/* + * 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.ml.tree.randomforest.data.impurity; + +import java.util.Optional; +import org.apache.ignite.ml.dataset.feature.Histogram; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; + +/** + * Interface represents an object that can compute best splitting point using features histograms. + * + * @param Base object type for histogram. + * @param Type of histogram that can be used in math operations with this object. + */ +public interface ImpurityComputer> extends Histogram { + /** + * Returns best split point computed on histogram if it exists. + * Split point may be absent when there is no data in histograms or split point lay in last bucket in histogram. + * + * @return Splitting point for decision tree. + */ + public Optional findBestSplit(); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogram.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogram.java new file mode 100644 index 0000000000000..296d862e02b06 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogram.java @@ -0,0 +1,74 @@ +/* + * 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.ml.tree.randomforest.data.impurity; + +import java.util.Optional; +import java.util.Set; +import java.util.TreeSet; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; + +/** + * Helper class for ImpurityHistograms. + */ +public abstract class ImpurityHistogram { + /** Bucket ids. */ + protected final Set bucketIds = new TreeSet<>(); + + /** Feature id. */ + protected final int featureId; + + /** + * Creates an instance of ImpurityHistogram. + * + * @param featureId Feature id. + */ + public ImpurityHistogram(int featureId) { + this.featureId = featureId; + } + + /** + * Checks split value validity and return Optional-wrap of it. + * In other case returns Optional.empty + * + * @param bestBucketId Best bucket id. + * @param bestSplitVal Best split value. + * @param bestImpurity Best impurity. + * @return best split value. + */ + protected Optional checkAndReturnSplitValue(int bestBucketId, double bestSplitVal, double bestImpurity) { + if (isLastBucket(bestBucketId)) + return Optional.empty(); + else + return Optional.of(new NodeSplit(featureId, bestSplitVal, bestImpurity)); + } + + /** + * @param bestBucketId Best bucket id. + * @return true if best found bucket is last within all bucketIds. + */ + private boolean isLastBucket(int bestBucketId) { + int minBucketId = Integer.MAX_VALUE; + int maxBucketId = Integer.MIN_VALUE; + for (Integer bucketId : bucketIds) { + minBucketId = Math.min(minBucketId, bucketId); + maxBucketId = Math.max(maxBucketId, bucketId); + } + + return bestBucketId == maxBucketId; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java new file mode 100644 index 0000000000000..d1ed87f1a94b9 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramsComputer.java @@ -0,0 +1,211 @@ +/* + * 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.ml.tree.randomforest.data.impurity; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.feature.BucketMeta; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedDatasetPartition; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.randomforest.data.NodeId; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; +import org.apache.ignite.ml.tree.randomforest.data.TreeNode; +import org.apache.ignite.ml.tree.randomforest.data.TreeRoot; + +/** + * Class containing logic of aggregation impurity statistics within learning dataset. + * + * @param Type of basic impurity computer for feature. + */ +public abstract class ImpurityHistogramsComputer> implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = -4984067145908187508L; + + /** + * Computes histograms for each features. + * + * @param roots Random forest roots. + * @param histMeta Histograms meta. + * @param nodesToLearn Nodes to learn. + * @param dataset Dataset. + */ + public Map> aggregateImpurityStatistics(ArrayList roots, + Map histMeta, Map nodesToLearn, + Dataset dataset) { + + return dataset.compute( + x -> aggregateImpurityStatisticsOnPartition(x, roots, histMeta, nodesToLearn), + this::reduceImpurityStatistics + ); + } + + /** + * Aggregates statistics for impurity computing for each corner nodes for each trees in random forest. This + * algorithm predict corner node in decision tree for learning vector and stocks it to correspond histogram. + * + * @param dataset Dataset. + * @param roots Trees. + * @param histMeta Histogram buckets meta. + * @param part Partition. + * @return Leaf statistics for impurity computing. + */ + private Map> aggregateImpurityStatisticsOnPartition( + BootstrappedDatasetPartition dataset, ArrayList roots, + Map histMeta, + Map part) { + + Map> res = part.keySet().stream() + .collect(Collectors.toMap(n -> n, NodeImpurityHistograms::new)); + + dataset.forEach(vector -> { + for (int sampleId = 0; sampleId < vector.counters().length; sampleId++) { + if (vector.counters()[sampleId] == 0) + continue; + + TreeRoot root = roots.get(sampleId); + NodeId key = root.getRootNode().predictNextNodeKey(vector.features()); + if (!part.containsKey(key)) //if we didn't take all nodes from learning queue + continue; + + NodeImpurityHistograms statistics = res.get(key); + for (Integer featureId : root.getUsedFeatures()) { + BucketMeta meta = histMeta.get(featureId); + if (!statistics.perFeatureStatistics.containsKey(featureId)) + statistics.perFeatureStatistics.put(featureId, createImpurityComputerForFeature(sampleId, meta)); + S impurityComputer = statistics.perFeatureStatistics.get(featureId); + impurityComputer.addElement(vector); + } + } + }); + return res; + } + + /** + * Merge leaf statistics from several data partitions. + * + * @param left Left. + * @param right Right. + * @return merged leaf impurity statistics. + */ + private Map> reduceImpurityStatistics(Map> left, + Map> right) { + + if (left == null) + return right; + if (right == null) + return left; + + Map> res = new HashMap<>(left); + for (NodeId key : right.keySet()) { + NodeImpurityHistograms rightVal = right.get(key); + if (!res.containsKey(key)) + res.put(key, rightVal); + else + res.put(key, left.get(key).plus(rightVal)); + } + + return res; + } + + /** + * Creates impurity computer in according to specific algorithm based on random forest (for example {@link + * GiniHistogram} for classification). + * + * @param sampleId Sample id. + * @param meta Bucket Meta. + * @return impurity computer + */ + protected abstract S createImpurityComputerForFeature(int sampleId, BucketMeta meta); + + /** + * Class represents per feature statistics for impurity computing. + */ + public static class NodeImpurityHistograms> implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 2700045747590421768L; + + /** Node id. */ + private final NodeId nodeId; + + /** Per feature statistics. */ + private final Map perFeatureStatistics = new HashMap<>(); + + /** + * Create an instance of NodeImpurityHistograms. + * + * @param nodeId Node id. + */ + public NodeImpurityHistograms(NodeId nodeId) { + this.nodeId = nodeId; + } + + /** + * Store features statistics from other instance. + * + * @param other Other instance. + */ + public NodeImpurityHistograms plus(NodeImpurityHistograms other) { + assert nodeId == other.nodeId; + NodeImpurityHistograms res = new NodeImpurityHistograms<>(nodeId); + addTo(this.perFeatureStatistics, res.perFeatureStatistics); + addTo(other.perFeatureStatistics, res.perFeatureStatistics); + return res; + } + + /** + * Adds all statistics to target. + * + * @param from From. + * @param to To. + */ + private void addTo(Map from, Map to) { + from.forEach((key, hist) -> { + if(!to.containsKey(key)) { + to.put(key, hist); + } else { + S sumOfHists = to.get(key).plus(hist); + to.put(key, sumOfHists); + } + }); + } + + /** */ + public NodeId getNodeId() { + return nodeId; + } + + /** + * Find best split point, based on feature statistics. + * + * @return Best split point if it exists. + */ + public Optional findBestSplit() { + return perFeatureStatistics.values().stream() + .flatMap(x -> x.findBestSplit().map(Stream::of).orElse(Stream.empty())) + .min(Comparator.comparingDouble(NodeSplit::getImpurity)); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogram.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogram.java new file mode 100644 index 0000000000000..e015e3f883c35 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogram.java @@ -0,0 +1,235 @@ +/* + * 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.ml.tree.randomforest.data.impurity; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.TreeMap; +import org.apache.ignite.ml.dataset.feature.BucketMeta; +import org.apache.ignite.ml.dataset.feature.ObjectHistogram; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; + +/** + * Class contains implementation of splitting point finding algorithm based on MSE metric (see https://en.wikipedia.org/wiki/Mean_squared_error) + * and represents a set of histograms in according to this metric. + */ +public class MSEHistogram extends ImpurityHistogram implements ImpurityComputer { + /** Serial version uid. */ + private static final long serialVersionUID = 9175485616887867623L; + + /** Bucket meta. */ + private final BucketMeta bucketMeta; + + /** Sample id. */ + private final int sampleId; + + /** Counters. */ + private ObjectHistogram counters; + + /** Sums of label values. */ + private ObjectHistogram sumOfLabels; + + /** Sums of squared label values. */ + private ObjectHistogram sumOfSquaredLabels; + + /** + * Creates an instance of MSEHistogram. + * + * @param sampleId Sample id. + * @param bucketMeta Bucket meta. + */ + public MSEHistogram(int sampleId, BucketMeta bucketMeta) { + super(bucketMeta.getFeatureMeta().getFeatureId()); + this.bucketMeta = bucketMeta; + this.sampleId = sampleId; + + counters = new ObjectHistogram<>(this::bucketMap, this::counterMap); + sumOfLabels = new ObjectHistogram<>(this::bucketMap, this::ysMap); + sumOfSquaredLabels = new ObjectHistogram<>(this::bucketMap, this::y2sMap); + } + + /** {@inheritDoc} */ + @Override public void addElement(BootstrappedVector vector) { + counters.addElement(vector); + sumOfLabels.addElement(vector); + sumOfSquaredLabels.addElement(vector); + } + + /** {@inheritDoc} */ + @Override public MSEHistogram plus(MSEHistogram other) { + MSEHistogram res = new MSEHistogram(sampleId, bucketMeta); + res.counters = this.counters.plus(other.counters); + res.sumOfLabels = this.sumOfLabels.plus(other.sumOfLabels); + res.sumOfSquaredLabels = this.sumOfSquaredLabels.plus(other.sumOfSquaredLabels); + res.bucketIds.addAll(this.bucketIds); + res.bucketIds.addAll(bucketIds); + return res; + } + + /** {@inheritDoc} */ + @Override public Set buckets() { + return bucketIds; + } + + /** {@inheritDoc} */ + @Override public Optional getValue(Integer bucketId) { + throw new NotImplementedException(); + } + + /** {@inheritDoc} */ + @Override public Optional findBestSplit() { + double bestImpurity = Double.POSITIVE_INFINITY; + double bestSplitVal = Double.NEGATIVE_INFINITY; + int bestBucketId = -1; + + //counter corresponds to number of samples + //ys corresponds to sumOfLabels + //y2s corresponds to sumOfSquaredLabels + TreeMap cntrDistrib = counters.computeDistributionFunction(); + TreeMap ysDistrib = sumOfLabels.computeDistributionFunction(); + TreeMap y2sDistrib = sumOfSquaredLabels.computeDistributionFunction(); + + double cntrMax = cntrDistrib.lastEntry().getValue(); + double ysMax = ysDistrib.lastEntry().getValue(); + double y2sMax = y2sDistrib.lastEntry().getValue(); + + double lastLeftCntrVal = 0.0; + double lastLeftYVal = 0.0; + double lastLeftY2Val = 0.0; + + for (Integer bucketId : bucketIds) { + //values for impurity computing to the left of bucket value + double leftCnt = cntrDistrib.getOrDefault(bucketId, lastLeftCntrVal); + double leftY = ysDistrib.getOrDefault(bucketId, lastLeftYVal); + double leftY2 = y2sDistrib.getOrDefault(bucketId, lastLeftY2Val); + + //values for impurity computing to the right of bucket value + double rightCnt = cntrMax - leftCnt; + double rightY = ysMax - leftY; + double rightY2 = y2sMax - leftY2; + + double impurity = 0.0; + + if (leftCnt > 0) + impurity += impurity(leftCnt, leftY, leftY2); + if (rightCnt > 0) + impurity += impurity(rightCnt, rightY, rightY2); + + if (impurity < bestImpurity) { + bestImpurity = impurity; + bestSplitVal = bucketMeta.bucketIdToValue(bucketId); + bestBucketId = bucketId; + } + } + + return checkAndReturnSplitValue(bestBucketId, bestSplitVal, bestImpurity); + } + + /** + * Computes impurity function value. + * + * @param cnt Counter value. + * @param ys plus of Ys. + * @param y2s plus of Y^2 s. + * @return impurity value. + */ + private double impurity(double cnt, double ys, double y2s) { + return y2s - 2.0 * ys / cnt * ys + Math.pow(ys / cnt, 2) * cnt; + } + + /** + * Maps vector to bucket id. + * + * @param vec Vector. + * @return Bucket id. + */ + private Integer bucketMap(BootstrappedVector vec) { + int bucketId = bucketMeta.getBucketId(vec.features().get(featureId)); + this.bucketIds.add(bucketId); + return bucketId; + } + + /** + * Maps vector to counter value. + * + * @param vec Vector. + * @return Counter value. + */ + private Double counterMap(BootstrappedVector vec) { + return (double)vec.counters()[sampleId]; + } + + /** + * Maps vector to Y-value. + * + * @param vec Vector. + * @return Y value. + */ + private Double ysMap(BootstrappedVector vec) { + return vec.counters()[sampleId] * vec.label(); + } + + /** + * Maps vector to Y^2 value. + * + * @param vec Vec. + * @return Y^2 value. + */ + private Double y2sMap(BootstrappedVector vec) { + return vec.counters()[sampleId] * Math.pow(vec.label(), 2); + } + + /** + * @return Counters histogram. + */ + ObjectHistogram getCounters() { + return counters; + } + + /** + * @return Ys histogram. + */ + ObjectHistogram getSumOfLabels() { + return sumOfLabels; + } + + /** + * @return Y^2s histogram. + */ + ObjectHistogram getSumOfSquaredLabels() { + return sumOfSquaredLabels; + } + + /** {@inheritDoc} */ + @Override public boolean isEqualTo(MSEHistogram other) { + HashSet unionBuckets = new HashSet<>(buckets()); + unionBuckets.addAll(other.bucketIds); + if(unionBuckets.size() != bucketIds.size()) + return false; + + if(!this.counters.isEqualTo(other.counters)) + return false; + if(!this.sumOfLabels.isEqualTo(other.sumOfLabels)) + return false; + + return this.sumOfSquaredLabels.isEqualTo(other.sumOfSquaredLabels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramComputer.java new file mode 100644 index 0000000000000..412cfc91ed902 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramComputer.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.ml.tree.randomforest.data.impurity; + +import org.apache.ignite.ml.dataset.feature.BucketMeta; + +/** + * Histogram computer realization for MSE impurity metric. + */ +public class MSEHistogramComputer extends ImpurityHistogramsComputer { + /** Serial version uid. */ + private static final long serialVersionUID = -1544823437437088334L; + + /** {@inheritDoc} */ + @Override protected MSEHistogram createImpurityComputerForFeature(int sampleId, BucketMeta meta) { + return new MSEHistogram(sampleId, meta); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/package-info.java new file mode 100644 index 0000000000000..62e22592ff02a --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/impurity/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 implementation of impurity computers based on histograms. + */ +package org.apache.ignite.ml.tree.randomforest.data.impurity; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/package-info.java new file mode 100644 index 0000000000000..0311845389f07 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/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. + */ + +/** + * + * Package contains helper data structures for random forest implementation. + */ +package org.apache.ignite.ml.tree.randomforest.data; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/ClassifierLeafValuesComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/ClassifierLeafValuesComputer.java new file mode 100644 index 0000000000000..64297ff1a57ab --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/ClassifierLeafValuesComputer.java @@ -0,0 +1,82 @@ +/* + * 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.ml.tree.randomforest.data.statistics; + +import java.util.Comparator; +import java.util.Map; +import org.apache.ignite.ml.dataset.feature.ObjectHistogram; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; + +/** + * Implementation of {@link LeafValuesComputer} for classification task. + */ +public class ClassifierLeafValuesComputer extends LeafValuesComputer> { + /** Serial version uid. */ + private static final long serialVersionUID = 420416095877577599L; + + /** Label mapping. */ + private final Map lblMapping; + + /** + * Creates an instance of ClassifierLeafValuesComputer. + * + * @param lblMapping Label mapping. + */ + public ClassifierLeafValuesComputer(Map lblMapping) { + this.lblMapping = lblMapping; + } + + /** {@inheritDoc} */ + @Override protected void addElementToLeafStatistic(ObjectHistogram leafStatAggr, BootstrappedVector vec, int sampleId) { + leafStatAggr.addElement(vec); + } + + /** {@inheritDoc} */ + @Override protected ObjectHistogram mergeLeafStats(ObjectHistogram leftStats, + ObjectHistogram rightStats) { + + return leftStats.plus(rightStats); + } + + /** {@inheritDoc} */ + @Override protected ObjectHistogram createLeafStatsAggregator(int sampleId) { + return new ObjectHistogram<>( + x -> lblMapping.get(x.label()), + x -> (double)x.counters()[sampleId] + ); + } + + /** + * Returns the most frequent value in according to statistic. + * + * @param stat Leaf statistics. + */ + @Override protected double computeLeafValue(ObjectHistogram stat) { + Integer bucketId = stat.buckets().stream() + .max(Comparator.comparing(b -> stat.getValue(b).orElse(0.0))) + .orElse(-1); + + if(bucketId == -1) + return Double.NaN; + + return lblMapping.entrySet().stream() + .filter(x -> x.getValue().equals(bucketId)) + .findFirst() + .get().getKey(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/LeafValuesComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/LeafValuesComputer.java new file mode 100644 index 0000000000000..056eeced7b9c8 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/LeafValuesComputer.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.randomforest.data.statistics; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedDatasetPartition; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.randomforest.data.NodeId; +import org.apache.ignite.ml.tree.randomforest.data.TreeNode; +import org.apache.ignite.ml.tree.randomforest.data.TreeRoot; + +/** + * Class containing logic of leaf values computing after building of all trees in random forest. + * + * @param Type of leaf statistic. + */ +public abstract class LeafValuesComputer implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = -429848953091775832L; + + /** + * Takes a list of all built trees and in one map-reduceImpurityStatistics step collect statistics for evaluating + * leaf-values for each tree and sets values for leaves. + * + * @param roots Learned trees. + * @param dataset Dataset. + */ + public void setValuesForLeaves(ArrayList roots, + Dataset dataset) { + + Map leafs = roots.stream() + .flatMap(r -> r.getLeafs().stream()) + .collect(Collectors.toMap(TreeNode::getId, Function.identity())); + + Map stats = dataset.compute( + data -> computeLeafsStatisticsInPartition(roots, leafs, data), + this::mergeLeafStatistics + ); + + leafs.forEach((id, leaf) -> { + T stat = stats.get(id); + if(stat != null) { + double leafVal = computeLeafValue(stat); + leaf.setValue(leafVal); + } + }); + } + + /** + * Aggregates statistics on labels from learning dataset for each leaf nodes. + * + * @param roots Learned trees. + * @param leafs List of all leafs. + * @param data Data. + * @return statistics on labels for each leaf nodes. + */ + private Map computeLeafsStatisticsInPartition(ArrayList roots, + Map leafs, BootstrappedDatasetPartition data) { + + Map res = new HashMap<>(); + for (int sampleId = 0; sampleId < roots.size(); sampleId++) { + final int sampleIdConst = sampleId; + + data.forEach(vec -> { + NodeId leafId = roots.get(sampleIdConst).getRootNode().predictNextNodeKey(vec.features()); + if (!leafs.containsKey(leafId)) + throw new IllegalStateException(); + + if (!res.containsKey(leafId)) + res.put(leafId, createLeafStatsAggregator(sampleIdConst)); + + addElementToLeafStatistic(res.get(leafId), vec, sampleIdConst); + }); + } + + return res; + } + + /** + * Merges statistics on labels from several partitions. + * + * @param left first partition. + * @param right second partition. + * @return merged statistics. + */ + private Map mergeLeafStatistics(Map left, Map right) { + if (left == null) + return right; + if (right == null) + return left; + + Set keys = new HashSet<>(left.keySet()); + keys.addAll(right.keySet()); + for (NodeId key : keys) { + if (!left.containsKey(key)) + left.put(key, right.get(key)); + else if (right.containsKey(key)) + left.put(key, mergeLeafStats(left.get(key), right.get(key))); + } + + return left; + } + + /** + * Save vector to leaf statistic. + * + * @param leafStatAggr Leaf statistics aggregator. + * @param vec Vector. + * @param sampleId Sample id. + */ + protected abstract void addElementToLeafStatistic(T leafStatAggr, BootstrappedVector vec, int sampleId); + + /** + * Merge statistics for same leafs. + * + * @param leftStats First leaf stat aggregator. + * @param rightStats Second leaf stat aggregator. + */ + protected abstract T mergeLeafStats(T leftStats, T rightStats); + + /** + * Creates an instance of leaf statistics aggregator in according to concrete algorithm based on RandomForest. + * + * @param sampleId Sample id. + */ + protected abstract T createLeafStatsAggregator(int sampleId); + + /** + * Compute value from leaf based on statistics on labels corresponds to leaf. + * + * @param stat Leaf statistics. + */ + protected abstract double computeLeafValue(T stat); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/MeanValueStatistic.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/MeanValueStatistic.java new file mode 100644 index 0000000000000..c2c61d7a93633 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/MeanValueStatistic.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.ml.tree.randomforest.data.statistics; + +import java.io.Serializable; + +/** + * Statistics for mean value computing container. + */ +public class MeanValueStatistic implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = -6265792209142062174L; + + /** Sum of values. */ + private double sumOfValues; + + /** Count of values. */ + private long cntOfValues; + + /** + * Creates an instance of MeanValueStatistic. + * + * @param sumOfValues Sum of values. + * @param cntOfValues Count of values. + */ + public MeanValueStatistic(double sumOfValues, long cntOfValues) { + this.sumOfValues = sumOfValues; + this.cntOfValues = cntOfValues; + } + + /** + * @return mean value. + */ + public double mean() { + return sumOfValues / cntOfValues; + } + + /** */ + public double getSumOfValues() { + return sumOfValues; + } + + /** */ + public void setSumOfValues(double sumOfValues) { + this.sumOfValues = sumOfValues; + } + + /** */ + public long getCntOfValues() { + return cntOfValues; + } + + /** */ + public void setCntOfValues(long cntOfValues) { + this.cntOfValues = cntOfValues; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatistics.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatistics.java new file mode 100644 index 0000000000000..83412049cc649 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatistics.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.ml.tree.randomforest.data.statistics; + +import java.io.Serializable; + +/** + * Aggregator of normal distribution statistics for continual features. + */ +public class NormalDistributionStatistics implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = -5422805289301484436L; + + /** Min value. */ + private final double min; + + /** Max value. */ + private final double max; + + /** Sum of value squares. */ + private final double sumOfSquares; + + /** Sum of values. */ + private final double sumOfValues; + + /** Count of objects. */ + private final long n; + + /** + * Creates an instance of NormalDistributionStatistics. + * + * @param min Min. + * @param max Max. + * @param sumOfSquares Sum of squares. + * @param sumOfValues Sum of values. + * @param n N. + */ + public NormalDistributionStatistics(double min, double max, double sumOfSquares, double sumOfValues, long n) { + this.min = min; + this.max = max; + this.sumOfSquares = sumOfSquares; + this.sumOfValues = sumOfValues; + this.n = n; + } + + /** + * Returns plus of normal distribution statistics. + * + * @param stats Stats. + * @return plus of normal distribution statistics. + */ + public NormalDistributionStatistics plus(NormalDistributionStatistics stats) { + return new NormalDistributionStatistics( + Math.min(this.min, stats.min), + Math.max(this.max, stats.max), + this.sumOfSquares + stats.sumOfSquares, + this.sumOfValues + stats.sumOfValues, + this.n + stats.n + ); + } + + /** + * @return mean value. + */ + public double mean() { + return sumOfValues / n; + } + + /** + * @return variance value. + */ + public double variance() { + double mean = mean(); + return (sumOfSquares / n) - mean * mean; + } + + /** + * @return standard deviation value. + */ + public double std() { + return Math.sqrt(variance()); + } + + /** + * @return min value. + */ + public double min() { + return min; + } + + /** + * @return max value. + */ + public double max() { + return max; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputer.java new file mode 100644 index 0000000000000..d972631e53e4a --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputer.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.ml.tree.randomforest.data.statistics; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedDatasetPartition; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Normal distribution parameters computer logic. + */ +public class NormalDistributionStatisticsComputer implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = -3699071003012595743L; + + /** + * Computes statistics of normal distribution on features in dataset. + * + * @param meta Meta. + * @param dataset Dataset. + */ + public List computeStatistics(List meta, Dataset dataset) { + + return dataset.compute( + x -> computeStatsOnPartition(x, meta), + (l, r) -> reduceStats(l, r, meta) + ); + } + + /** + * Aggregates normal distribution statistics for continual features in dataset partition. + * + * @param part Partition. + * @param meta Meta. + * @return Statistics for each feature. + */ + public List computeStatsOnPartition(BootstrappedDatasetPartition part, + List meta) { + + double[] sumOfValues = new double[meta.size()]; + double[] sumOfSquares = new double[sumOfValues.length]; + double[] min = new double[sumOfValues.length]; + double[] max = new double[sumOfValues.length]; + Arrays.fill(min, Double.POSITIVE_INFINITY); + Arrays.fill(max, Double.NEGATIVE_INFINITY); + + for (int i = 0; i < part.getRowsCount(); i++) { + Vector vec = part.getRow(i).features(); + for (int featureId = 0; featureId < vec.size(); featureId++) { + if (!meta.get(featureId).isCategoricalFeature()) { + double featureVal = vec.get(featureId); + sumOfValues[featureId] += featureVal; + sumOfSquares[featureId] += Math.pow(featureVal, 2); + min[featureId] = Math.min(min[featureId], featureVal); + max[featureId] = Math.max(max[featureId], featureVal); + } + } + } + + ArrayList res = new ArrayList<>(); + for (int featureId = 0; featureId < sumOfSquares.length; featureId++) { + res.add(new NormalDistributionStatistics( + min[featureId], max[featureId], + sumOfSquares[featureId], sumOfValues[featureId], + part.getRowsCount()) + ); + } + return res; + } + + /** + * Merges statistics on features from two partitions. + * + * @param left Left. + * @param right Right. + * @param meta Features meta. + * @return plus of statistics for each features. + */ + public List reduceStats(List left, + List right, + List meta) { + + if (left == null) + return right; + if (right == null) + return left; + + assert meta.size() == left.size() && meta.size() == right.size(); + List res = new ArrayList<>(); + for (int featureId = 0; featureId < meta.size(); featureId++) { + NormalDistributionStatistics leftStat = left.get(featureId); + NormalDistributionStatistics rightStat = right.get(featureId); + res.add(leftStat.plus(rightStat)); + } + return res; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/RegressionLeafValuesComputer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/RegressionLeafValuesComputer.java new file mode 100644 index 0000000000000..7dcb02595a385 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/RegressionLeafValuesComputer.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.randomforest.data.statistics; + +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; + +/** + * Implementation of {@link LeafValuesComputer} for regression task. + */ +public class RegressionLeafValuesComputer extends LeafValuesComputer { + /** Serial version uid. */ + private static final long serialVersionUID = -1898031675220962125L; + + /** {@inheritDoc} */ + @Override protected void addElementToLeafStatistic(MeanValueStatistic leafStatAggr, + BootstrappedVector vec, int sampleId) { + + int numOfRepetitions = vec.counters()[sampleId]; + leafStatAggr.setSumOfValues(leafStatAggr.getSumOfValues() + vec.label() * numOfRepetitions); + leafStatAggr.setCntOfValues(leafStatAggr.getCntOfValues() + numOfRepetitions); + } + + /** {@inheritDoc} */ + @Override protected MeanValueStatistic mergeLeafStats(MeanValueStatistic leftStats, + MeanValueStatistic rightStats) { + + return new MeanValueStatistic( + leftStats.getSumOfValues() + rightStats.getSumOfValues(), + leftStats.getCntOfValues() + rightStats.getCntOfValues() + ); + } + + /** {@inheritDoc} */ + @Override protected MeanValueStatistic createLeafStatsAggregator(int sampleId) { + return new MeanValueStatistic(0.0, 0); + } + + /** + * Returns the mean value in according to statistic. + * + * @param stat Leaf statistics. + */ + @Override protected double computeLeafValue(MeanValueStatistic stat) { + return stat.mean(); + } + +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/package-info.java new file mode 100644 index 0000000000000..dff897047e142 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/data/statistics/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 implementation of statistics computers for Random Forest. + */ +package org.apache.ignite.ml.tree.randomforest.data.statistics; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java index 52ba70531c1c4..babddfb168c00 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/DatasetTestSuite.java @@ -17,6 +17,7 @@ package org.apache.ignite.ml.dataset; +import org.apache.ignite.ml.dataset.feature.ObjectHistogramTest; import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilderTest; import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetTest; import org.apache.ignite.ml.dataset.impl.cache.util.ComputeUtilsTest; @@ -42,7 +43,8 @@ CacheBasedDatasetTest.class, LocalDatasetBuilderTest.class, SimpleDatasetTest.class, - SimpleLabeledDatasetTest.class + SimpleLabeledDatasetTest.class, + ObjectHistogramTest.class }) public class DatasetTestSuite { // No-op. diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/feature/ObjectHistogramTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/feature/ObjectHistogramTest.java new file mode 100644 index 0000000000000..131b69babfc2c --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/feature/ObjectHistogramTest.java @@ -0,0 +1,170 @@ +/* + * 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.ml.dataset.feature; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.TreeMap; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** */ +public class ObjectHistogramTest { + /** Data first partition. */ + private double[] dataFirstPart = new double[] {0., 0., 0., 0., 1., 1., 1, 2., 2., 3., 4., 5.}; + /** Data second partition. */ + private double[] dataSecondPart = new double[] {0., 1., 0., 1., 0., 1., 0, 1., 0., 1., 0., 5., 6.}; + + /** */ + private ObjectHistogram hist1; + /** */ + private ObjectHistogram hist2; + + /** + * + */ + @Before + public void setUp() throws Exception { + hist1 = new ObjectHistogram<>(this::computeBucket, x -> 1.); + hist2 = new ObjectHistogram<>(this::computeBucket, x -> 1.); + + fillHist(hist1, dataFirstPart); + fillHist(hist2, dataSecondPart); + } + + /** + * @param hist History. + * @param data Data. + */ + private void fillHist(ObjectHistogram hist, double[] data) { + for (int i = 0; i < data.length; i++) + hist.addElement(data[i]); + } + + /** + * + */ + @Test + public void testBuckets() { + testBuckets(hist1, new int[] {0, 1, 2, 3, 4, 5}, new int[] {4, 3, 2, 1, 1, 1}); + testBuckets(hist2, new int[] {0, 1, 5, 6}, new int[] {6, 5, 1, 1}); + } + + /** + * @param hist History. + * @param expectedBuckets Expected buckets. + * @param expectedCounters Expected counters. + */ + private void testBuckets(ObjectHistogram hist, int[] expectedBuckets, int[] expectedCounters) { + int size = hist.buckets().size(); + int[] buckets = new int[size]; + int[] counters = new int[size]; + int ptr = 0; + for (int bucket : hist.buckets()) { + counters[ptr] = hist.getValue(bucket).get().intValue(); + buckets[ptr++] = bucket; + } + + assertArrayEquals(expectedBuckets, buckets); + assertArrayEquals(expectedCounters, counters); + } + + /** + * + */ + @Test + public void testAdd() { + double value = 100.; + hist1.addElement(value); + Optional counter = hist1.getValue(computeBucket(value)); + + assertTrue(counter.isPresent()); + assertEquals(1, counter.get().intValue()); + } + + /** + * + */ + @Test + public void testAddHist() { + ObjectHistogram result = hist1.plus(hist2); + testBuckets(result, new int[] {0, 1, 2, 3, 4, 5, 6}, new int[] {10, 8, 2, 1, 1, 2, 1}); + } + + /** + * + */ + @Test + public void testDistributionFunction() { + TreeMap distribution = hist1.computeDistributionFunction(); + + int[] buckets = new int[distribution.size()]; + double[] sums = new double[distribution.size()]; + + int ptr = 0; + for(int bucket : distribution.keySet()) { + sums[ptr] = distribution.get(bucket); + buckets[ptr++] = bucket; + } + + assertArrayEquals(new int[] {0, 1, 2, 3, 4, 5}, buckets); + assertArrayEquals(new double[] {4., 7., 9., 10., 11., 12.}, sums, 0.01); + } + + @Test + public void testOfSum() { + IgniteFunction bucketMap = x -> (int) (Math.ceil(x * 100) % 100); + IgniteFunction counterMap = x -> Math.pow(x, 2); + + ObjectHistogram forAllHistogram = new ObjectHistogram<>(bucketMap, counterMap); + Random rnd = new Random(); + List> partitions = new ArrayList<>(); + int cntOfPartitions = rnd.nextInt(100); + int sizeOfDataset = rnd.nextInt(10000); + for(int i = 0; i < cntOfPartitions; i++) + partitions.add(new ObjectHistogram<>(bucketMap, counterMap)); + + for(int i = 0; i < sizeOfDataset; i++) { + double objVal = rnd.nextDouble(); + forAllHistogram.addElement(objVal); + partitions.get(rnd.nextInt(partitions.size())).addElement(objVal); + } + + Optional> leftSum = partitions.stream().reduce((x,y) -> x.plus(y)); + Optional> rightSum = partitions.stream().reduce((x,y) -> y.plus(x)); + assertTrue(leftSum.isPresent()); + assertTrue(rightSum.isPresent()); + assertTrue(forAllHistogram.isEqualTo(leftSum.get())); + assertTrue(forAllHistogram.isEqualTo(rightSum.get())); + assertTrue(leftSum.get().isEqualTo(rightSum.get())); + } + + /** + * @param value Value. + */ + private int computeBucket(Double value) { + return (int)Math.rint(value); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java index ae9f4904670ff..636c60372fcf4 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java @@ -26,7 +26,6 @@ @RunWith(Suite.class) @Suite.SuiteClasses({ LearningEnvironmentBuilderTest.class, - LearningEnvironmentTest.class, PromiseTest.class }) public class EnvironmentTestSuite { diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java deleted file mode 100644 index 9f8bab7c38ab3..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java +++ /dev/null @@ -1,187 +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.ml.environment; - -import java.util.Arrays; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicReference; -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.QueryCursor; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.composition.ModelsComposition; -import org.apache.ignite.ml.environment.logging.ConsoleLogger; -import org.apache.ignite.ml.environment.logging.MLLogger; -import org.apache.ignite.ml.environment.parallelism.ParallelismStrategy; -import org.apache.ignite.ml.math.primitives.vector.VectorUtils; -import org.apache.ignite.ml.tree.randomforest.RandomForestRegressionTrainer; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.apache.ignite.thread.IgniteThread; - -/** - * Tests for {@link LearningEnvironment} that require to start the whole Ignite infrastructure. IMPL NOTE based on - * RandomForestRegressionExample example. - */ -public class LearningEnvironmentTest extends GridCommonAbstractTest { - /** Number of nodes in grid */ - private static final int NODE_COUNT = 1; - - /** Ignite instance. */ - private Ignite ignite; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() { - stopAllGrids(); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() { - /* Grid instance. */ - ignite = grid(NODE_COUNT); - ignite.configuration().setPeerClassLoadingEnabled(true); - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - } - - /** */ - public void testBasic() throws InterruptedException { - AtomicReference actualAmount = new AtomicReference<>(null); - AtomicReference actualMse = new AtomicReference<>(null); - AtomicReference actualMae = new AtomicReference<>(null); - - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - LearningEnvironmentTest.class.getSimpleName(), () -> { - IgniteCache dataCache = getTestCache(ignite); - - RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer(13, 4, 101, 0.3, 2, 0); - - trainer.setEnvironment(LearningEnvironment.builder() - .withParallelismStrategy(ParallelismStrategy.Type.ON_DEFAULT_POOL) - .withLoggingFactory(ConsoleLogger.factory(MLLogger.VerboseLevel.LOW)) - .build() - ); - - ModelsComposition randomForest = trainer.fit(ignite, dataCache, - (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), - (k, v) -> v[v.length - 1] - ); - - double mse = 0.0; - double mae = 0.0; - int totalAmount = 0; - - try (QueryCursor> observations = dataCache.query(new ScanQuery<>())) { - for (Cache.Entry observation : observations) { - double difference = estimatePrediction(randomForest, observation); - - mse += Math.pow(difference, 2.0); - mae += Math.abs(difference); - - totalAmount++; - } - } - - actualAmount.set(totalAmount); - - mse = mse / totalAmount; - actualMse.set(mse); - - mae = mae / totalAmount; - actualMae.set(mae); - }); - - igniteThread.start(); - igniteThread.join(); - - assertEquals("Total amount", 23, (int)actualAmount.get()); - assertTrue("Mean squared error (MSE)", actualMse.get() > 0); - assertTrue("Mean absolute error (MAE)", actualMae.get() > 0); - } - - /** */ - private double estimatePrediction(ModelsComposition randomForest, Cache.Entry observation) { - double[] val = observation.getValue(); - double[] inputs = Arrays.copyOfRange(val, 0, val.length - 1); - double groundTruth = val[val.length - 1]; - - double prediction = randomForest.apply(VectorUtils.of(inputs)); - - return prediction - groundTruth; - } - - /** - * Fills cache with data and returns it. - * - * @param ignite Ignite instance. - * @return Filled Ignite Cache. - */ - private IgniteCache getTestCache(Ignite ignite) { - CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); - cacheConfiguration.setName(UUID.randomUUID().toString()); - cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10)); - - IgniteCache cache = ignite.createCache(cacheConfiguration); - - for (int i = 0; i < data.length; i++) - cache.put(i, data[i]); - - return cache; - } - - /** - * Part of the Boston housing dataset. - */ - private static final double[][] data = { - {0.02731,0.00,7.070,0,0.4690,6.4210,78.90,4.9671,2,242.0,17.80,396.90,9.14,21.60}, - {0.02729,0.00,7.070,0,0.4690,7.1850,61.10,4.9671,2,242.0,17.80,392.83,4.03,34.70}, - {0.03237,0.00,2.180,0,0.4580,6.9980,45.80,6.0622,3,222.0,18.70,394.63,2.94,33.40}, - {0.06905,0.00,2.180,0,0.4580,7.1470,54.20,6.0622,3,222.0,18.70,396.90,5.33,36.20}, - {0.02985,0.00,2.180,0,0.4580,6.4300,58.70,6.0622,3,222.0,18.70,394.12,5.21,28.70}, - {0.08829,12.50,7.870,0,0.5240,6.0120,66.60,5.5605,5,311.0,15.20,395.60,12.43,22.90}, - {0.14455,12.50,7.870,0,0.5240,6.1720,96.10,5.9505,5,311.0,15.20,396.90,19.15,27.10}, - {0.21124,12.50,7.870,0,0.5240,5.6310,100.00,6.0821,5,311.0,15.20,386.63,29.93,16.50}, - {0.17004,12.50,7.870,0,0.5240,6.0040,85.90,6.5921,5,311.0,15.20,386.71,17.10,18.90}, - {0.22489,12.50,7.870,0,0.5240,6.3770,94.30,6.3467,5,311.0,15.20,392.52,20.45,15.00}, - {0.11747,12.50,7.870,0,0.5240,6.0090,82.90,6.2267,5,311.0,15.20,396.90,13.27,18.90}, - {0.09378,12.50,7.870,0,0.5240,5.8890,39.00,5.4509,5,311.0,15.20,390.50,15.71,21.70}, - {0.62976,0.00,8.140,0,0.5380,5.9490,61.80,4.7075,4,307.0,21.00,396.90,8.26,20.40}, - {0.63796,0.00,8.140,0,0.5380,6.0960,84.50,4.4619,4,307.0,21.00,380.02,10.26,18.20}, - {0.62739,0.00,8.140,0,0.5380,5.8340,56.50,4.4986,4,307.0,21.00,395.62,8.47,19.90}, - {1.05393,0.00,8.140,0,0.5380,5.9350,29.30,4.4986,4,307.0,21.00,386.85,6.58,23.10}, - {0.78420,0.00,8.140,0,0.5380,5.9900,81.70,4.2579,4,307.0,21.00,386.75,14.67,17.50}, - {0.80271,0.00,8.140,0,0.5380,5.4560,36.60,3.7965,4,307.0,21.00,288.99,11.69,20.20}, - {0.72580,0.00,8.140,0,0.5380,5.7270,69.50,3.7965,4,307.0,21.00,390.95,11.28,18.20}, - {1.25179,0.00,8.140,0,0.5380,5.5700,98.10,3.7979,4,307.0,21.00,376.57,21.02,13.60}, - {0.85204,0.00,8.140,0,0.5380,5.9650,89.20,4.0123,4,307.0,21.00,392.53,13.83,19.60}, - {1.23247,0.00,8.140,0,0.5380,6.1420,91.70,3.9769,4,307.0,21.00,396.90,18.72,15.20}, - {0.98843,0.00,8.140,0,0.5380,5.8130,100.00,4.0952,4,307.0,21.00,394.54,19.88,14.50} - }; - -} - diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeTestSuite.java index 867103eabc45c..2cbb486c3df5f 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeTestSuite.java @@ -24,8 +24,6 @@ import org.apache.ignite.ml.tree.impurity.mse.MSEImpurityMeasureTest; import org.apache.ignite.ml.tree.impurity.util.SimpleStepFunctionCompressorTest; import org.apache.ignite.ml.tree.impurity.util.StepFunctionTest; -import org.apache.ignite.ml.tree.randomforest.RandomForestClassifierTrainerTest; -import org.apache.ignite.ml.tree.randomforest.RandomForestRegressionTrainerTest; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -44,9 +42,7 @@ MSEImpurityMeasureCalculatorTest.class, MSEImpurityMeasureTest.class, StepFunctionTest.class, - SimpleStepFunctionCompressorTest.class, - RandomForestClassifierTrainerTest.class, - RandomForestRegressionTrainerTest.class + SimpleStepFunctionCompressorTest.class }) public class DecisionTreeTestSuite { } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java index d06fa507b6fe5..4abf5083170cf 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java @@ -21,11 +21,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.ignite.ml.composition.ModelOnFeaturesSubspace; import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.composition.predictionsaggregator.OnMajorityPredictionsAggregator; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; -import org.apache.ignite.ml.tree.DecisionTreeConditionalNode; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -75,26 +74,15 @@ public void testFit() { sample.put(new double[] {x1, x2, x3, x4}, (double)(i % 2)); } - RandomForestClassifierTrainer trainer = new RandomForestClassifierTrainer(4, 3, 5, 0.3, 4, 0.1) - .withUseIndex(false); + ArrayList meta = new ArrayList<>(); + for(int i = 0; i < 4; i++) + meta.add(new FeatureMeta("", i, false)); + RandomForestClassifierTrainer trainer = new RandomForestClassifierTrainer(meta) + .withCountOfTrees(5) + .withFeaturesCountSelectionStrgy(x -> 2); ModelsComposition mdl = trainer.fit(sample, parts, (k, v) -> VectorUtils.of(k), (k, v) -> v); - mdl.getModels().forEach(m -> { - assertTrue(m instanceof ModelOnFeaturesSubspace); - - ModelOnFeaturesSubspace mdlOnFeaturesSubspace = (ModelOnFeaturesSubspace) m; - - assertTrue(mdlOnFeaturesSubspace.getMdl() instanceof DecisionTreeConditionalNode); - - assertTrue(mdlOnFeaturesSubspace.getFeaturesMapping().size() > 0); - - String expClsName = "ModelOnFeatureSubspace"; - assertTrue(mdlOnFeaturesSubspace.toString().contains(expClsName)); - assertTrue(mdlOnFeaturesSubspace.toString(true).contains(expClsName)); - assertTrue(mdlOnFeaturesSubspace.toString(false).contains(expClsName)); - }); - assertTrue(mdl.getPredictionsAggregator() instanceof OnMajorityPredictionsAggregator); assertEquals(5, mdl.getModels().size()); } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java index 987176e4b3174..c4a4a750db604 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java @@ -21,11 +21,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.ignite.ml.composition.ModelOnFeaturesSubspace; import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.composition.predictionsaggregator.MeanValuePredictionsAggregator; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; -import org.apache.ignite.ml.tree.DecisionTreeConditionalNode; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -72,16 +71,14 @@ public void testFit() { sample.put(x1 * x2 + x3 * x4, new double[] {x1, x2, x3, x4}); } - RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer(4, 3, 5, 0.3, 4, 0.1) - .withUseIndex(false); + ArrayList meta = new ArrayList<>(); + for(int i = 0; i < 4; i++) + meta.add(new FeatureMeta("", i, false)); + RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer(meta) + .withCountOfTrees(5) + .withFeaturesCountSelectionStrgy(x -> 2); ModelsComposition mdl = trainer.fit(sample, parts, (k, v) -> VectorUtils.of(v), (k, v) -> k); - - mdl.getModels().forEach(m -> { - assertTrue(m instanceof ModelOnFeaturesSubspace); - assertTrue(((ModelOnFeaturesSubspace) m).getMdl() instanceof DecisionTreeConditionalNode); - }); - assertTrue(mdl.getPredictionsAggregator() instanceof MeanValuePredictionsAggregator); assertEquals(5, mdl.getModels().size()); } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.java new file mode 100644 index 0000000000000..ed474fe9429fe --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.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.ml.tree.randomforest; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; +import org.apache.ignite.ml.tree.randomforest.data.TreeNode; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** */ +public class RandomForestTest { + /** Seed. */ + private final long seed = 0; + + /** Count of trees. */ + private final int countOfTrees = 10; + + /** Min imp delta. */ + private final double minImpDelta = 1.0; + + /** Max depth. */ + private final int maxDepth = 1; + + /** Meta. */ + private final List meta = Arrays.asList( + new FeatureMeta("", 0, false), + new FeatureMeta("", 1, true), + new FeatureMeta("", 2, false), + new FeatureMeta("", 3, true), + new FeatureMeta("", 4, false), + new FeatureMeta("", 5, true), + new FeatureMeta("", 6, false) + ); + + /** Rf. */ + private RandomForestClassifierTrainer rf = new RandomForestClassifierTrainer(meta) + .withCountOfTrees(countOfTrees) + .withSeed(seed) + .withFeaturesCountSelectionStrgy(x -> 4) + .withMaxDepth(maxDepth) + .withMinImpurityDelta(minImpDelta) + .withSubsampleSize(0.1); + + /** */ + @Test + public void testNeedSplit() { + TreeNode node = new TreeNode(1, 1); + node.setImpurity(1000); + assertTrue(rf.needSplit(node, Optional.of(new NodeSplit(0, 0, node.getImpurity() - minImpDelta * 1.01)))); + assertFalse(rf.needSplit(node, Optional.of(new NodeSplit(0, 0, node.getImpurity() - minImpDelta * 0.5)))); + assertFalse(rf.needSplit(node, Optional.of(new NodeSplit(0, 0, node.getImpurity())))); + + TreeNode child = node.toConditional(0, 0).get(0); + child.setImpurity(1000); + assertFalse(rf.needSplit(child, Optional.of(new NodeSplit(0, 0, child.getImpurity() - minImpDelta * 1.01)))); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTreeTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTreeTestSuite.java new file mode 100644 index 0000000000000..cc51352361b31 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTreeTestSuite.java @@ -0,0 +1,39 @@ +/* + * 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.ml.tree.randomforest; + +import org.apache.ignite.ml.tree.randomforest.data.impurity.GiniFeatureHistogramTest; +import org.apache.ignite.ml.tree.randomforest.data.impurity.MSEHistogramTest; +import org.apache.ignite.ml.tree.randomforest.data.statistics.NormalDistributionStatisticsComputerTest; +import org.junit.runner.RunWith; +import org.junit.runners.Suite; + +/** + * Test suite for all tests located in {@link org.apache.ignite.ml.tree} package. + */ +@RunWith(Suite.class) +@Suite.SuiteClasses({ + RandomForestClassifierTrainerTest.class, + RandomForestRegressionTrainerTest.class, + GiniFeatureHistogramTest.class, + MSEHistogramTest.class, + NormalDistributionStatisticsComputerTest.class, + RandomForestTest.class +}) +public class RandomForestTreeTestSuite { +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/TreeNodeTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/TreeNodeTest.java new file mode 100644 index 0000000000000..943b5d8d2e038 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/TreeNodeTest.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.ml.tree.randomforest.data; + +import java.util.List; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** */ +public class TreeNodeTest { + /** Features 1. */ + private final Vector features1 = VectorUtils.of(0., 1.); + /** Features 2. */ + private final Vector features2 = VectorUtils.of(1., 0.); + + /** */ + @Test + public void testPredictNextIdCondNodeAtTreeCorner() { + TreeNode node = new TreeNode(5, 1); + + assertEquals(TreeNode.Type.UNKNOWN, node.getType()); + assertEquals(5, node.predictNextNodeKey(features1).nodeId()); + assertEquals(5, node.predictNextNodeKey(features2).nodeId()); + } + + /** */ + @Test + public void testPredictNextIdForLeaf() { + TreeNode node = new TreeNode(5, 1); + node.toLeaf(0.5); + + assertEquals(TreeNode.Type.LEAF, node.getType()); + assertEquals(5, node.predictNextNodeKey(features1).nodeId()); + assertEquals(5, node.predictNextNodeKey(features2).nodeId()); + } + + /** */ + @Test + public void testPredictNextIdForTree() { + TreeNode root = new TreeNode(1, 1); + root.toConditional(0, 0.1); + + assertEquals(TreeNode.Type.CONDITIONAL, root.getType()); + assertEquals(2, root.predictNextNodeKey(features1).nodeId()); + assertEquals(3, root.predictNextNodeKey(features2).nodeId()); + } + + /** */ + @Test + public void testPredictProba() { + TreeNode root = new TreeNode(1, 1); + List leaves = root.toConditional(0, 0.1); + leaves.forEach(leaf -> { + leaf.toLeaf(leaf.getId().nodeId() % 2); + }); + + assertEquals(TreeNode.Type.CONDITIONAL, root.getType()); + assertEquals(0.0, root.apply(features1), 0.001); + assertEquals(1.0, root.apply(features2), 0.001); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniFeatureHistogramTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniFeatureHistogramTest.java new file mode 100644 index 0000000000000..7ca6411a14b12 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/GiniFeatureHistogramTest.java @@ -0,0 +1,254 @@ +/* + * 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.ml.tree.randomforest.data.impurity; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.ignite.ml.dataset.feature.BucketMeta; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.apache.ignite.ml.tree.randomforest.data.NodeSplit; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** */ +public class GiniFeatureHistogramTest extends ImpurityHistogramTest { + /** Feature 1 meta. */ + private BucketMeta feature1Meta = new BucketMeta(new FeatureMeta("", 0, true)); + /** Feature 2 meta. */ + private BucketMeta feature2Meta = new BucketMeta(new FeatureMeta("", 1, false)); + /** Feature 3 meta. */ + private BucketMeta feature3Meta = new BucketMeta(new FeatureMeta("", 2, true)); + + /** */ + @Before + public void setUp() throws Exception { + feature2Meta.setMinVal(-5); + feature2Meta.setBucketSize(1); + } + + /** */ + @Test + public void testAddVector() { + Map lblMapping = new HashMap<>(); + lblMapping.put(1.0, 0); + lblMapping.put(2.0, 1); + lblMapping.put(3.0, 2); + + GiniHistogram catFeatureSmpl1 = new GiniHistogram(0, lblMapping, feature1Meta); + GiniHistogram catFeatureSmpl2 = new GiniHistogram(1, lblMapping, feature1Meta); + + GiniHistogram contFeatureSmpl1 = new GiniHistogram(0, lblMapping, feature2Meta); + GiniHistogram contFeatureSmpl2 = new GiniHistogram(1, lblMapping, feature2Meta); + + for (BootstrappedVector vec : dataset) { + catFeatureSmpl1.addElement(vec); + catFeatureSmpl2.addElement(vec); + contFeatureSmpl1.addElement(vec); + contFeatureSmpl2.addElement(vec); + } + + checkBucketIds(catFeatureSmpl1.buckets(), new Integer[] {0, 1}); + checkBucketIds(catFeatureSmpl2.buckets(), new Integer[] {0, 1}); + checkBucketIds(contFeatureSmpl1.buckets(), new Integer[] {1, 4, 6, 7, 8}); + checkBucketIds(contFeatureSmpl2.buckets(), new Integer[] {1, 4, 6, 7, 8}); + + //categorical feature + checkCounters(catFeatureSmpl1.getHistForLabel(1.0), new double[] {2, 1}); //for feature values 0 and 1 + checkBucketIds(catFeatureSmpl1.getHistForLabel(1.0).buckets(), new Integer[] {0, 1}); + checkCounters(catFeatureSmpl1.getHistForLabel(2.0), new double[] {3}); //for feature value 1 + checkBucketIds(catFeatureSmpl1.getHistForLabel(2.0).buckets(), new Integer[] {1}); + checkCounters(catFeatureSmpl1.getHistForLabel(3.0), new double[] {2}); //for feature value 0 + checkBucketIds(catFeatureSmpl1.getHistForLabel(3.0).buckets(), new Integer[] {0}); + + checkCounters(catFeatureSmpl2.getHistForLabel(1.0), new double[] {1, 2}); //for feature values 0 and 1 + checkBucketIds(catFeatureSmpl2.getHistForLabel(1.0).buckets(), new Integer[] {0, 1}); + checkCounters(catFeatureSmpl2.getHistForLabel(2.0), new double[] {3}); //for feature value 1 + checkBucketIds(catFeatureSmpl2.getHistForLabel(2.0).buckets(), new Integer[] {1}); + checkCounters(catFeatureSmpl2.getHistForLabel(3.0), new double[] {0}); //for feature value 0 + checkBucketIds(catFeatureSmpl2.getHistForLabel(3.0).buckets(), new Integer[] {0}); + + //continuous feature + checkCounters(contFeatureSmpl1.getHistForLabel(1.0), new double[] {1, 2}); //for feature values 0 and 1 + checkBucketIds(contFeatureSmpl1.getHistForLabel(1.0).buckets(), new Integer[] {4, 6}); + checkCounters(contFeatureSmpl1.getHistForLabel(2.0), new double[] {1, 2}); //for feature value 1 + checkBucketIds(contFeatureSmpl1.getHistForLabel(2.0).buckets(), new Integer[] {1, 7}); + checkCounters(contFeatureSmpl1.getHistForLabel(3.0), new double[] {2}); //for feature value 0 + checkBucketIds(contFeatureSmpl1.getHistForLabel(3.0).buckets(), new Integer[] {8}); + + checkCounters(contFeatureSmpl2.getHistForLabel(1.0), new double[] {2, 1}); //for feature values 0 and 1 + checkBucketIds(contFeatureSmpl2.getHistForLabel(1.0).buckets(), new Integer[] {4, 6}); + checkCounters(contFeatureSmpl2.getHistForLabel(2.0), new double[] {2, 1}); //for feature value 1 + checkBucketIds(contFeatureSmpl2.getHistForLabel(2.0).buckets(), new Integer[] {1, 7}); + checkCounters(contFeatureSmpl2.getHistForLabel(3.0), new double[] {0}); //for feature value 0 + checkBucketIds(contFeatureSmpl2.getHistForLabel(3.0).buckets(), new Integer[] {8}); + } + + /** */ + @Test + public void testSplit() { + Map lblMapping = new HashMap<>(); + lblMapping.put(1.0, 0); + lblMapping.put(2.0, 1); + + GiniHistogram catFeatureSmpl1 = new GiniHistogram(0, lblMapping, feature1Meta); + GiniHistogram contFeatureSmpl1 = new GiniHistogram(0, lblMapping, feature2Meta); + GiniHistogram emptyHist = new GiniHistogram(0, lblMapping, feature3Meta); + GiniHistogram catFeatureSmpl2 = new GiniHistogram(0, lblMapping, feature3Meta); + + feature2Meta.setMinVal(-5); + feature2Meta.setBucketSize(1); + + for (BootstrappedVector vec : toSplitDataset) { + catFeatureSmpl1.addElement(vec); + contFeatureSmpl1.addElement(vec); + catFeatureSmpl2.addElement(vec); + } + + NodeSplit catSplit = catFeatureSmpl1.findBestSplit().get(); + NodeSplit contSplit = contFeatureSmpl1.findBestSplit().get(); + assertEquals(1.0, catSplit.getValue(), 0.01); + assertEquals(-0.5, contSplit.getValue(), 0.01); + assertFalse(emptyHist.findBestSplit().isPresent()); + assertFalse(catFeatureSmpl2.findBestSplit().isPresent()); + } + + @Test + public void testOfSums() { + int sampleId = 0; + BucketMeta bucketMeta1 = new BucketMeta(new FeatureMeta("", 0, false)); + bucketMeta1.setMinVal(0.); + bucketMeta1.setBucketSize(0.1); + BucketMeta bucketMeta2 = new BucketMeta(new FeatureMeta("", 1, true)); + + GiniHistogram forAllHist1 = new GiniHistogram(sampleId, lblMapping, bucketMeta1); + GiniHistogram forAllHist2 = new GiniHistogram(sampleId, lblMapping, bucketMeta2); + + List partitions1 = new ArrayList<>(); + List partitions2 = new ArrayList<>(); + int countOfPartitions = rnd.nextInt(1000); + for(int i = 0; i < countOfPartitions; i++) { + partitions1.add(new GiniHistogram(sampleId,lblMapping, bucketMeta1)); + partitions2.add(new GiniHistogram(sampleId,lblMapping, bucketMeta2)); + } + + int datasetSize = rnd.nextInt(10000); + for(int i = 0; i < datasetSize; i++) { + BootstrappedVector vec = randomVector(2, 1, true); + vec.features().set(1, (vec.features().get(1) * 100) % 100); + + forAllHist1.addElement(vec); + forAllHist2.addElement(vec); + int partitionId = rnd.nextInt(countOfPartitions); + partitions1.get(partitionId).addElement(vec); + partitions2.get(partitionId).addElement(vec); + } + + checkSums(forAllHist1, partitions1); + checkSums(forAllHist2, partitions2); + + GiniHistogram emptyHist1 = new GiniHistogram(sampleId, lblMapping, bucketMeta1); + GiniHistogram emptyHist2 = new GiniHistogram(sampleId, lblMapping, bucketMeta2); + assertTrue(forAllHist1.isEqualTo(forAllHist1.plus(emptyHist1))); + assertTrue(forAllHist2.isEqualTo(forAllHist2.plus(emptyHist2))); + assertTrue(forAllHist1.isEqualTo(emptyHist1.plus(forAllHist1))); + assertTrue(forAllHist2.isEqualTo(emptyHist2.plus(forAllHist2))); + } + + /** */ + @Test + public void testJoin() { + Map lblMapping = new HashMap<>(); + lblMapping.put(1.0, 0); + lblMapping.put(2.0, 1); + lblMapping.put(3.0, 2); + + GiniHistogram catFeatureSmpl1 = new GiniHistogram(0, lblMapping, feature1Meta); + GiniHistogram catFeatureSmpl2 = new GiniHistogram(0, lblMapping, feature1Meta); + + GiniHistogram contFeatureSmpl1 = new GiniHistogram(0, lblMapping, feature2Meta); + GiniHistogram contFeatureSmpl2 = new GiniHistogram(0, lblMapping, feature2Meta); + + for (BootstrappedVector vec : dataset) { + catFeatureSmpl1.addElement(vec); + contFeatureSmpl1.addElement(vec); + } + + for (BootstrappedVector vec : toSplitDataset) { + catFeatureSmpl2.addElement(vec); + contFeatureSmpl2.addElement(vec); + } + + GiniHistogram res1 = catFeatureSmpl1.plus(catFeatureSmpl2); + GiniHistogram res2 = contFeatureSmpl1.plus(contFeatureSmpl2); + + checkBucketIds(res1.buckets(), new Integer[] {0, 1, 2}); + checkBucketIds(res2.buckets(), new Integer[] {1, 4, 6, 7, 8}); + + //categorical feature + checkCounters(res1.getHistForLabel(1.0), new double[] {3, 2, 6}); //for feature values 0 and 1 + checkBucketIds(res1.getHistForLabel(1.0).buckets(), new Integer[] {0, 1, 2}); + checkCounters(res1.getHistForLabel(2.0), new double[] {4, 6}); //for feature value 1 + checkBucketIds(res1.getHistForLabel(2.0).buckets(), new Integer[] {0, 1}); + checkCounters(res1.getHistForLabel(3.0), new double[] {2}); //for feature value 0 + checkBucketIds(res1.getHistForLabel(3.0).buckets(), new Integer[] {0}); + + //continuous feature + checkCounters(res2.getHistForLabel(1.0), new double[] {1, 1, 8, 1}); //for feature values 0 and 1 + checkBucketIds(res2.getHistForLabel(1.0).buckets(), new Integer[] {1, 4, 6, 8}); + checkCounters(res2.getHistForLabel(2.0), new double[] {1, 4, 0, 5}); //for feature value 1 + checkBucketIds(res2.getHistForLabel(2.0).buckets(), new Integer[] {1, 4, 6, 7}); + checkCounters(res2.getHistForLabel(3.0), new double[] {2}); //for feature value 0 + checkBucketIds(res2.getHistForLabel(3.0).buckets(), new Integer[] {8}); + } + + /** Dataset. */ + private BootstrappedVector[] dataset = new BootstrappedVector[] { + new BootstrappedVector(VectorUtils.of(1, -1), 1, new int[] {1, 2}), + new BootstrappedVector(VectorUtils.of(1, 2), 2, new int[] {2, 1}), + new BootstrappedVector(VectorUtils.of(0, 3), 3, new int[] {2, 0}), + new BootstrappedVector(VectorUtils.of(0, 1), 1, new int[] {2, 1}), + new BootstrappedVector(VectorUtils.of(1, -4), 2, new int[] {1, 2}), + }; + + /** To split dataset. */ + private BootstrappedVector[] toSplitDataset = new BootstrappedVector[] { + new BootstrappedVector(VectorUtils.of(0, -1, 0, 0), 2, new int[] {2}), + new BootstrappedVector(VectorUtils.of(0, -1, 0, 0), 2, new int[] {1}), + new BootstrappedVector(VectorUtils.of(0, -1, 0, 0), 2, new int[] {1}), + new BootstrappedVector(VectorUtils.of(0, 3, 0, 0), 1, new int[] {1}), + new BootstrappedVector(VectorUtils.of(0, 1, 0, 0), 2, new int[] {0}), + new BootstrappedVector(VectorUtils.of(1, 2, 0, 0), 2, new int[] {1}), + new BootstrappedVector(VectorUtils.of(1, 2, 0, 0), 2, new int[] {1}), + new BootstrappedVector(VectorUtils.of(1, 2, 0, 0), 2, new int[] {1}), + new BootstrappedVector(VectorUtils.of(1, -4, 0, 0), 1, new int[] {1}), + new BootstrappedVector(VectorUtils.of(2, 1, 0, 0), 1, new int[] {1}), + new BootstrappedVector(VectorUtils.of(2, 1, 0, 0), 1, new int[] {1}), + new BootstrappedVector(VectorUtils.of(2, 1, 0, 0), 1, new int[] {1}), + new BootstrappedVector(VectorUtils.of(2, 1, 0, 0), 1, new int[] {1}), + new BootstrappedVector(VectorUtils.of(2, 1, 0, 0), 1, new int[] {1}), + new BootstrappedVector(VectorUtils.of(2, 1, 0, 1), 1, new int[] {1}), + }; +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramTest.java new file mode 100644 index 0000000000000..df4c15464699c --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/ImpurityHistogramTest.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.randomforest.data.impurity; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.stream.DoubleStream; +import java.util.stream.IntStream; +import org.apache.ignite.ml.dataset.feature.Histogram; +import org.apache.ignite.ml.dataset.feature.ObjectHistogram; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertTrue; + +public class ImpurityHistogramTest { + protected static final int COUNT_OF_CLASSES = 3; + protected static final Map lblMapping = new HashMap<>(); + protected Random rnd = new Random(); + + static { + for(int i = 0; i < COUNT_OF_CLASSES; i++) + lblMapping.put((double)i, i); + } + + protected void checkBucketIds(Set bucketIdsSet, Integer[] expected) { + Integer[] bucketIds = new Integer[bucketIdsSet.size()]; + bucketIdsSet.toArray(bucketIds); + assertArrayEquals(expected, bucketIds); + } + + protected void checkCounters(ObjectHistogram hist, double[] expected) { + double[] counters = hist.buckets().stream().mapToDouble(x -> hist.getValue(x).get()).toArray(); + assertArrayEquals(expected, counters, 0.01); + } + + protected BootstrappedVector randomVector(int countOfFeatures, int countOfSampes, boolean isClassification) { + double[] features = DoubleStream.generate(() -> rnd.nextDouble()).limit(countOfFeatures).toArray(); + int[] counters = IntStream.generate(() -> rnd.nextInt(10)).limit(countOfSampes).toArray(); + double lbl = isClassification ? Math.abs(rnd.nextInt() % COUNT_OF_CLASSES) : rnd.nextDouble(); + return new BootstrappedVector(VectorUtils.of(features), lbl, counters); + } + + protected > void checkSums(T expected, List partitions) { + T leftSum = partitions.stream().reduce((x,y) -> x.plus(y)).get(); + T rightSum = partitions.stream().reduce((x,y) -> y.plus(x)).get(); + assertTrue(expected.isEqualTo(leftSum)); + assertTrue(expected.isEqualTo(rightSum)); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramTest.java new file mode 100644 index 0000000000000..41bd5ff09de17 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/impurity/MSEHistogramTest.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.randomforest.data.impurity; + +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.ml.dataset.feature.BucketMeta; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +/** */ +public class MSEHistogramTest extends ImpurityHistogramTest { + /** Feature 1 meta. */ + private BucketMeta feature1Meta = new BucketMeta(new FeatureMeta("", 0, true)); + /** Feature 2 meta. */ + private BucketMeta feature2Meta = new BucketMeta(new FeatureMeta("", 1, false)); + + /** */ + @Before + public void setUp() throws Exception { + feature2Meta.setMinVal(-5); + feature2Meta.setBucketSize(1); + } + + /** */ + @Test + public void testAdd() { + MSEHistogram catHist1 = new MSEHistogram(0, feature1Meta); + MSEHistogram contHist1 = new MSEHistogram(0, feature2Meta); + + MSEHistogram catHist2 = new MSEHistogram(1, feature1Meta); + MSEHistogram contHist2 = new MSEHistogram(1, feature2Meta); + + for (BootstrappedVector vec : dataset) { + catHist1.addElement(vec); + catHist2.addElement(vec); + contHist1.addElement(vec); + contHist2.addElement(vec); + } + + checkBucketIds(catHist1.buckets(), new Integer[] {0, 1}); + checkBucketIds(catHist2.buckets(), new Integer[] {0, 1}); + checkBucketIds(contHist1.buckets(), new Integer[] {1, 4, 6, 7, 8}); + checkBucketIds(contHist2.buckets(), new Integer[] {1, 4, 6, 7, 8}); + + //counters + checkCounters(catHist1.getCounters(), new double[] {4, 4}); + checkCounters(catHist2.getCounters(), new double[] {1, 5}); + checkCounters(contHist1.getCounters(), new double[] {1, 1, 2, 2, 2}); + checkCounters(contHist2.getCounters(), new double[] {2, 2, 1, 1, 0}); + + //ys + checkCounters(catHist1.getSumOfLabels(), new double[] {2 * 4 + 2 * 3, 5 + 1 + 2 * 2}); + checkCounters(catHist2.getSumOfLabels(), new double[] {4, 2 * 5 + 2 * 1 + 2}); + checkCounters(contHist1.getSumOfLabels(), new double[] {5 * 1, 1 * 1, 4 * 2, 2 * 2, 3 * 2}); + checkCounters(contHist2.getSumOfLabels(), new double[]{ 2 * 5, 2 * 1, 1 * 4, 2 * 1, 0 * 3 }); + + //y2s + checkCounters(catHist1.getSumOfSquaredLabels(), new double[] {2 * 4 * 4 + 2 * 3 * 3, 5 * 5 + 1 + 2 * 2 * 2}); + checkCounters(catHist2.getSumOfSquaredLabels(), new double[] {4 * 4, 2 * 5 * 5 + 2 * 1 * 1 + 2 * 2}); + checkCounters(contHist1.getSumOfSquaredLabels(), new double[] {1 * 5 * 5, 1 * 1 * 1, 2 * 4 * 4, 2 * 2 * 2, 2 * 3 * 3}); + checkCounters(contHist2.getSumOfSquaredLabels(), new double[]{ 2 * 5 * 5, 2 * 1 * 1, 1 * 4 * 4, 1 * 2 * 2, 0 * 3 * 3 }); + } + + @Test + public void testOfSums() { + int sampleId = 0; + BucketMeta bucketMeta1 = new BucketMeta(new FeatureMeta("", 0, false)); + bucketMeta1.setMinVal(0.); + bucketMeta1.setBucketSize(0.1); + BucketMeta bucketMeta2 = new BucketMeta(new FeatureMeta("", 1, true)); + + MSEHistogram forAllHist1 = new MSEHistogram(sampleId, bucketMeta1); + MSEHistogram forAllHist2 = new MSEHistogram(sampleId, bucketMeta2); + + List partitions1 = new ArrayList<>(); + List partitions2 = new ArrayList<>(); + int countOfPartitions = rnd.nextInt(100); + for(int i = 0; i < countOfPartitions; i++) { + partitions1.add(new MSEHistogram(sampleId, bucketMeta1)); + partitions2.add(new MSEHistogram(sampleId, bucketMeta2)); + } + + int datasetSize = rnd.nextInt(1000); + for(int i = 0; i < datasetSize; i++) { + BootstrappedVector vec = randomVector(2, 1, false); + vec.features().set(1, (vec.features().get(1) * 100) % 100); + + forAllHist1.addElement(vec); + forAllHist2.addElement(vec); + int partitionId = rnd.nextInt(countOfPartitions); + partitions1.get(partitionId).addElement(vec); + partitions2.get(partitionId).addElement(vec); + } + + checkSums(forAllHist1, partitions1); + checkSums(forAllHist2, partitions2); + + MSEHistogram emptyHist1 = new MSEHistogram(sampleId, bucketMeta1); + MSEHistogram emptyHist2 = new MSEHistogram(sampleId, bucketMeta2); + assertTrue(forAllHist1.isEqualTo(forAllHist1.plus(emptyHist1))); + assertTrue(forAllHist2.isEqualTo(forAllHist2.plus(emptyHist2))); + assertTrue(forAllHist1.isEqualTo(emptyHist1.plus(forAllHist1))); + assertTrue(forAllHist2.isEqualTo(emptyHist2.plus(forAllHist2))); + } + + /** Dataset. */ + private BootstrappedVector[] dataset = new BootstrappedVector[] { + new BootstrappedVector(VectorUtils.of(1, -4), 5, new int[] {1, 2}), + new BootstrappedVector(VectorUtils.of(1, -1), 1, new int[] {1, 2}), + new BootstrappedVector(VectorUtils.of(0, 1), 4, new int[] {2, 1}), + new BootstrappedVector(VectorUtils.of(1, 2), 2, new int[] {2, 1}), + new BootstrappedVector(VectorUtils.of(0, 3), 3, new int[] {2, 0}), + }; +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputerTest.java new file mode 100644 index 0000000000000..79ee3b6bf401b --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/data/statistics/NormalDistributionStatisticsComputerTest.java @@ -0,0 +1,130 @@ +/* + * 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.ml.tree.randomforest.data.statistics; + +import java.util.Arrays; +import java.util.List; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedDatasetPartition; +import org.apache.ignite.ml.dataset.impl.bootstrapping.BootstrappedVector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** */ +public class NormalDistributionStatisticsComputerTest { + /** Features Meta. */ + private final List meta = Arrays.asList( + new FeatureMeta("", 0, false), + new FeatureMeta("", 1, true), + new FeatureMeta("", 2, false), + new FeatureMeta("", 3, true), + new FeatureMeta("", 4, false), + new FeatureMeta("", 5, true), + new FeatureMeta("", 6, false) + ); + + /** Partition. */ + private BootstrappedDatasetPartition partition = new BootstrappedDatasetPartition(new BootstrappedVector[] { + new BootstrappedVector(VectorUtils.of(0, 1, 2, 1, 4, 2, 6), 0., null), + new BootstrappedVector(VectorUtils.of(1, 0, 3, 2, 5, 3, 7), 0., null), + new BootstrappedVector(VectorUtils.of(2, 1, 4, 1, 6, 2, 8), 0., null), + new BootstrappedVector(VectorUtils.of(3, 0, 5, 2, 7, 3, 9), 0., null), + new BootstrappedVector(VectorUtils.of(4, 1, 6, 1, 8, 2, 10), 0., null), + new BootstrappedVector(VectorUtils.of(5, 0, 7, 2, 9, 3, 11), 0., null), + new BootstrappedVector(VectorUtils.of(6, 1, 8, 1, 10, 2, 12), 0., null), + new BootstrappedVector(VectorUtils.of(7, 0, 9, 2, 11, 3, 13), 0., null), + new BootstrappedVector(VectorUtils.of(8, 1, 10, 1, 12, 2, 14), 0., null), + new BootstrappedVector(VectorUtils.of(9, 0, 11, 2, 13, 3, 15), 0., null), + }); + + private NormalDistributionStatisticsComputer computer = new NormalDistributionStatisticsComputer(); + + /** */ + @Test + public void computeStatsOnPartitionTest() { + List result = computer.computeStatsOnPartition(partition, meta); + NormalDistributionStatistics[] expected = new NormalDistributionStatistics[] { + new NormalDistributionStatistics(0, 9, 285, 45, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(2, 11, 505, 65, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(4, 13, 805, 85, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(6, 15, 1185, 105, 10), + }; + + assertEquals(expected.length, result.size()); + for (int i = 0; i < expected.length; i++) { + NormalDistributionStatistics expectedStat = expected[i]; + NormalDistributionStatistics resultStat = result.get(i); + assertEquals(expectedStat.mean(), resultStat.mean(), 0.01); + assertEquals(expectedStat.variance(), resultStat.variance(), 0.01); + assertEquals(expectedStat.std(), resultStat.std(), 0.01); + assertEquals(expectedStat.min(), resultStat.min(), 0.01); + assertEquals(expectedStat.max(), resultStat.max(), 0.01); + } + } + + /** */ + @Test + public void reduceStatsTest() { + List left = Arrays.asList( + new NormalDistributionStatistics(0, 9, 285, 45, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(2, 11, 505, 65, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(4, 13, 805, 85, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(6, 15, 1185, 105, 10) + ); + + List right = Arrays.asList( + new NormalDistributionStatistics(6, 15, 1185, 105, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(4, 13, 805, 85, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(2, 11, 505, 65, 10), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(0, 9, 285, 45, 10) + ); + + List result = computer.reduceStats(left, right, meta); + NormalDistributionStatistics[] expected = new NormalDistributionStatistics[] { + new NormalDistributionStatistics(0, 15, 1470, 150, 20), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(2, 13, 1310, 150, 20), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(2, 13, 1310, 150, 20), + new NormalDistributionStatistics(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0, 0, 10), + new NormalDistributionStatistics(0, 15, 1470, 150, 20) + }; + + assertEquals(expected.length, result.size()); + for (int i = 0; i < expected.length; i++) { + NormalDistributionStatistics expectedStat = expected[i]; + NormalDistributionStatistics resultStat = result.get(i); + assertEquals(expectedStat.mean(), resultStat.mean(), 0.01); + assertEquals(expectedStat.variance(), resultStat.variance(), 0.01); + assertEquals(expectedStat.std(), resultStat.std(), 0.01); + assertEquals(expectedStat.min(), resultStat.min(), 0.01); + assertEquals(expectedStat.max(), resultStat.max(), 0.01); + } + } +} From 053f88d7d5be41f34f80c33c6689cac835d8892e Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 31 Aug 2018 00:15:28 +0300 Subject: [PATCH 23/95] IGNITE-9116 .NET: LINQ: Use CacheConfiguration.SqlSchema when generating SQL This closes #4642 --- .../ExpiryCacheHolderTest.cs | 5 -- .../IgniteSessionStateStoreProviderTest.cs | 7 +-- .../Cache/Query/Linq/CacheLinqTest.Base.cs | 20 +++++-- .../Query/Linq/CacheLinqTest.Introspection.cs | 22 +++---- .../IgniteStartStopTest.cs | 23 ++++---- .../Apache.Ignite.Core.Tests/MessagingTest.cs | 13 ++--- .../Apache.Ignite.Core.Tests/ReconnectTest.cs | 3 +- .../Apache.Ignite.Core/IgniteConfiguration.cs | 56 +++++++++--------- .../Impl/ExpressionWalker.cs | 58 +++++++++++++++++-- 9 files changed, 128 insertions(+), 79 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs index b3d52283e023d..eb2797787186f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs @@ -416,11 +416,6 @@ public Task GetSizeAsync(params CachePeekMode[] modes) throw new NotImplementedException(); } - public void LocalPromote(IEnumerable keys) - { - throw new NotImplementedException(); - } - public IQueryCursor> Query(QueryBase qry) { throw new NotImplementedException(); diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs index 08c44a6b0f261..da074acbcd76f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs @@ -23,7 +23,6 @@ namespace Apache.Ignite.AspNet.Tests using System.Linq; using System.Reflection; using System.Threading; - using System.Threading.Tasks; using System.Web; using System.Web.SessionState; using Apache.Ignite.Core; @@ -56,7 +55,7 @@ public class IgniteSessionStateStoreProviderTest private const string Id = "1"; /** Test context. */ - private static readonly HttpContext HttpContext = + private static readonly HttpContext HttpContext = new HttpContext(new HttpRequest(null, "http://tempuri.org", null), new HttpResponse(null)); /// @@ -87,7 +86,7 @@ public void TearDown() var ignite = Ignition.GetIgnite(GridName); ignite.GetCacheNames().ToList().ForEach(x => ignite.GetCache(x).RemoveAll()); } - + /// /// Test setup. /// @@ -230,7 +229,7 @@ public void TestCaching() // Add item. provider.CreateUninitializedItem(HttpContext, Id, 7); - + // Check added item. res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); Assert.IsNotNull(res); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Base.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Base.cs index 5b56abd9c4185..81dcfedf4db2b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Base.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Base.cs @@ -45,11 +45,17 @@ public partial class CacheLinqTest /** Cache name. */ private const string PersonOrgCacheName = "person_org"; + /** Cache schema. */ + private const string PersonOrgCacheSchema = "person_org_Schema"; + /** Cache name. */ private const string PersonSecondCacheName = "person_cache"; - /** Role cache name. */ - private const string RoleCacheName = "role_cache"; + /** Cache schema. */ + private const string PersonSecondCacheSchema = "\"person_cache_SCHEMA\""; + + /** Role cache name: uses invalid characters to test name escaping. */ + private const string RoleCacheName = "role$ cache."; /** */ private const int RoleCount = 3; @@ -199,7 +205,8 @@ public void FixtureTearDown() new QueryEntity(typeof (int), typeof (Organization))) { CacheMode = CacheMode.Replicated, - SqlEscapeAll = GetSqlEscapeAll() + SqlEscapeAll = GetSqlEscapeAll(), + SqlSchema = PersonOrgCacheSchema }); } @@ -231,14 +238,15 @@ public void FixtureTearDown() }) { CacheMode = CacheMode.Replicated, - SqlEscapeAll = GetSqlEscapeAll() + SqlEscapeAll = GetSqlEscapeAll(), + SqlSchema = PersonSecondCacheSchema }); } /// /// Checks that function maps to SQL function properly. /// - private static void CheckFunc(Expression> exp, IQueryable query, + private static void CheckFunc(Expression> exp, IQueryable query, Func localResultFunc = null) { localResultFunc = localResultFunc ?? (x => x); @@ -263,7 +271,7 @@ public void FixtureTearDown() /// /// Checks that function used in Where Clause maps to SQL function properly /// - private static void CheckWhereFunc(IQueryable> query, + private static void CheckWhereFunc(IQueryable> query, Expression,bool>> whereExpression) { // Calculate result locally, using real method invocation diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs index aa2654863789f..f5b5baa4f3569 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs @@ -66,8 +66,8 @@ public void TestIntrospection() Assert.AreEqual( GetSqlEscapeAll() - ? "select _T0._KEY, _T0._VAL from \"person_org\".\"Person\" as _T0 where (_T0.\"_KEY\" > ?)" - : "select _T0._KEY, _T0._VAL from \"person_org\".Person as _T0 where (_T0._KEY > ?)", + ? "select _T0._KEY, _T0._VAL from PERSON_ORG_SCHEMA.\"Person\" as _T0 where (_T0.\"_KEY\" > ?)" + : "select _T0._KEY, _T0._VAL from PERSON_ORG_SCHEMA.Person as _T0 where (_T0._KEY > ?)", fq.Sql); Assert.AreEqual(new[] { 10 }, fq.Arguments); @@ -84,12 +84,12 @@ public void TestIntrospection() var str = query.ToString(); Assert.AreEqual(GetSqlEscapeAll() ? "CacheQueryable [CacheName=person_org, TableName=Person, Query=SqlFieldsQuery " + - "[Sql=select _T0._KEY, _T0._VAL from \"person_org\".\"Person\" as _T0 where " + + "[Sql=select _T0._KEY, _T0._VAL from PERSON_ORG_SCHEMA.\"Person\" as _T0 where " + "(_T0.\"_KEY\" > ?), Arguments=[10], " + "Local=True, PageSize=999, EnableDistributedJoins=False, EnforceJoinOrder=True, " + "Timeout=00:00:02.5000000, ReplicatedOnly=True, Colocated=True, Schema=, Lazy=True]]" : "CacheQueryable [CacheName=person_org, TableName=Person, Query=SqlFieldsQuery " + - "[Sql=select _T0._KEY, _T0._VAL from \"person_org\".Person as _T0 where " + + "[Sql=select _T0._KEY, _T0._VAL from PERSON_ORG_SCHEMA.Person as _T0 where " + "(_T0._KEY > ?), Arguments=[10], " + "Local=True, PageSize=999, EnableDistributedJoins=False, EnforceJoinOrder=True, " + "Timeout=00:00:02.5000000, ReplicatedOnly=True, Colocated=True, Schema=, Lazy=True]]", str); @@ -104,8 +104,8 @@ public void TestIntrospection() fq = fieldsQuery.GetFieldsQuery(); Assert.AreEqual(GetSqlEscapeAll() - ? "select _T0.\"Name\" from \"person_org\".\"Person\" as _T0" - : "select _T0.NAME from \"person_org\".Person as _T0", + ? "select _T0.\"Name\" from PERSON_ORG_SCHEMA.\"Person\" as _T0" + : "select _T0.NAME from PERSON_ORG_SCHEMA.Person as _T0", fq.Sql); Assert.IsFalse(fq.Local); @@ -117,11 +117,11 @@ public void TestIntrospection() str = fieldsQuery.ToString(); Assert.AreEqual(GetSqlEscapeAll() ? "CacheQueryable [CacheName=person_org, TableName=Person, Query=SqlFieldsQuery " + - "[Sql=select _T0.\"Name\" from \"person_org\".\"Person\" as _T0, Arguments=[], Local=False, " + + "[Sql=select _T0.\"Name\" from PERSON_ORG_SCHEMA.\"Person\" as _T0, Arguments=[], Local=False, " + "PageSize=1024, EnableDistributedJoins=False, EnforceJoinOrder=False, " + "Timeout=00:00:00, ReplicatedOnly=False, Colocated=False, Schema=, Lazy=False]]" : "CacheQueryable [CacheName=person_org, TableName=Person, Query=SqlFieldsQuery " + - "[Sql=select _T0.NAME from \"person_org\".Person as _T0, Arguments=[], Local=False, " + + "[Sql=select _T0.NAME from PERSON_ORG_SCHEMA.Person as _T0, Arguments=[], Local=False, " + "PageSize=1024, EnableDistributedJoins=False, EnforceJoinOrder=False, " + "Timeout=00:00:00, ReplicatedOnly=False, Colocated=False, Schema=, Lazy=False]]", str); @@ -136,17 +136,17 @@ public void TestIntrospection() str = distrQuery.ToString(); Assert.AreEqual(GetSqlEscapeAll() ? "CacheQueryable [CacheName=person_org, TableName=Person, Query=SqlFieldsQuery " + - "[Sql=select _T0._KEY, _T0._VAL from \"person_org\".\"Person\" as _T0 where " + + "[Sql=select _T0._KEY, _T0._VAL from PERSON_ORG_SCHEMA.\"Person\" as _T0 where " + "(((_T0.\"_KEY\" > ?) and (_T0.\"age1\" > ?)) " + "and (_T0.\"Name\" like \'%\' || ? || \'%\') ), Arguments=[10, 20, x], Local=False, " + "PageSize=1024, EnableDistributedJoins=True, EnforceJoinOrder=False, " + "Timeout=00:00:00, ReplicatedOnly=False, Colocated=False, Schema=, Lazy=False]]" : "CacheQueryable [CacheName=person_org, TableName=Person, Query=SqlFieldsQuery " + - "[Sql=select _T0._KEY, _T0._VAL from \"person_org\".Person as _T0 where " + + "[Sql=select _T0._KEY, _T0._VAL from PERSON_ORG_SCHEMA.Person as _T0 where " + "(((_T0._KEY > ?) and (_T0.AGE1 > ?)) " + "and (_T0.NAME like \'%\' || ? || \'%\') ), Arguments=[10, 20, x], Local=False, " + "PageSize=1024, EnableDistributedJoins=True, EnforceJoinOrder=False, " + "Timeout=00:00:00, ReplicatedOnly=False, Colocated=False, Schema=, Lazy=False]]", str); } } -} \ No newline at end of file +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs index f9c1cad8055cd..dd6a7b2c1f172 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs @@ -15,13 +15,12 @@ * limitations under the License. */ -namespace Apache.Ignite.Core.Tests +namespace Apache.Ignite.Core.Tests { using System; using System.IO; using System.Linq; using System.Threading; - using System.Threading.Tasks; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Messaging; @@ -45,7 +44,7 @@ public void TearDown() } /// - /// + /// /// [Test] public void TestStartDefault() @@ -60,7 +59,7 @@ public void TestStartDefault() } /// - /// + /// /// [Test] public void TestStartWithConfigPath() @@ -79,7 +78,7 @@ public void TestStartWithConfigPath() } /// - /// + /// /// [Test] public void TestStartGetStop() @@ -140,7 +139,7 @@ public void TestStartGetStop() } /// - /// + /// /// [Test] public void TestStartTheSameName() @@ -148,7 +147,7 @@ public void TestStartTheSameName() var cfg = TestUtils.GetTestConfiguration(name: "grid1"); var grid1 = Ignition.Start(cfg); Assert.AreEqual("grid1", grid1.Name); - + var ex = Assert.Throws(() => Ignition.Start(cfg)); Assert.AreEqual("Ignite instance with this name has already been started: grid1", ex.Message); } @@ -173,7 +172,7 @@ public void TestStartUniqueName() } /// - /// + /// /// [Test] public void TestUsageAfterStop() @@ -191,7 +190,7 @@ public void TestUsageAfterStop() } /// - /// + /// /// [Test] public void TestStartStopLeak() @@ -244,7 +243,7 @@ public void TestClientMode() } } } - finally + finally { Ignition.ClientMode = false; } @@ -299,13 +298,13 @@ public void TestProcessorInit() "-jvmClasspath=" + TestUtils.CreateTestClasspath(), "-springConfigUrl=" + Path.GetFullPath(cfg.SpringConfigUrl), "-J-Xms512m", "-J-Xmx512m"); - + Assert.IsTrue(proc.Alive); var cts = new CancellationTokenSource(); var token = cts.Token; - // Spam message subscriptions on a separate thread + // Spam message subscriptions on a separate thread // to test race conditions during processor init on remote node var listenTask = TaskRunner.Run(() => { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs index 7db4eef09f44a..5dfa82fe917ae 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs @@ -23,7 +23,6 @@ namespace Apache.Ignite.Core.Tests using System.Diagnostics.CodeAnalysis; using System.Linq; using System.Threading; - using System.Threading.Tasks; using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; @@ -59,7 +58,7 @@ public sealed class MessagingTest DateTime.Now, byte.MinValue, short.MaxValue, - + // Enums. CacheMode.Local, GCCollectionMode.Forced, @@ -94,7 +93,7 @@ public void TearDown() MessagingTestHelper.AssertFailures(); } - finally + finally { // Stop all grids between tests to drop any hanging messages Ignition.StopAll(true); @@ -466,7 +465,7 @@ public void TestRemoteListenMultithreaded() if (sharedResult.Length != 0) { - Assert.Fail("Unexpected messages ({0}): {1}; last sent message: {2}", sharedResult.Length, + Assert.Fail("Unexpected messages ({0}): {1}; last sent message: {2}", sharedResult.Length, string.Join(",", sharedResult), lastMsg); } } @@ -574,7 +573,7 @@ public static class MessagingTestHelper { /** */ public static readonly ConcurrentStack ReceivedMessages = new ConcurrentStack(); - + /** */ private static readonly ConcurrentStack Failures = new ConcurrentStack(); @@ -626,7 +625,7 @@ public static void ClearReceived(int expectedCount) // check that all messages came from local node. var localNodeId = cluster.Ignite.GetCluster().GetLocalNode().Id; Assert.AreEqual(localNodeId, LastNodeIds.Distinct().Single()); - + AssertFailures(); } @@ -670,7 +669,7 @@ public bool Invoke(Guid nodeId, string message) } catch (Exception ex) { - // When executed on remote nodes, these exceptions will not go to sender, + // When executed on remote nodes, these exceptions will not go to sender, // so we have to accumulate them. Failures.Push(string.Format("Exception in Listen (msg: {0}, id: {1}): {2}", message, nodeId, ex)); throw; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs index 274439e4e3be6..5d40408c2bee7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs @@ -17,7 +17,6 @@ namespace Apache.Ignite.Core.Tests { - using System; using System.Threading; using Apache.Ignite.Core.Cache; using Apache.Ignite.Core.Cache.Configuration; @@ -83,7 +82,7 @@ public void TestClusterRestart() // Check reconnect task. Assert.IsTrue(ex.ClientReconnectTask.Result); - + // Wait a bit for notifications. Thread.Sleep(100); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index 0d66b9f78364e..55d358ab369f7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -529,7 +529,7 @@ internal void Write(BinaryWriter writer, ClientProtocolVersion srvVer) // SSL Context factory. SslFactorySerializer.Write(writer, SslContextFactory); - + // Failure handler. if (FailureHandler == null) { @@ -538,7 +538,7 @@ internal void Write(BinaryWriter writer, ClientProtocolVersion srvVer) else { writer.WriteBoolean(true); - + if (FailureHandler is NoOpFailureHandler) { writer.WriteByte(0); @@ -547,7 +547,7 @@ internal void Write(BinaryWriter writer, ClientProtocolVersion srvVer) { writer.WriteByte(1); } - else + else { var failHnd = FailureHandler as StopNodeOrHaltFailureHandler; @@ -565,7 +565,7 @@ internal void Write(BinaryWriter writer, ClientProtocolVersion srvVer) failHnd.Write(writer); } } - + // Plugins (should be last). if (PluginConfigurations != null) { @@ -806,7 +806,7 @@ private void ReadCore(BinaryReader r, ClientProtocolVersion srvVer) // SSL context factory. SslContextFactory = SslFactorySerializer.Read(r); - + //Failure handler. if (r.ReadBoolean()) { @@ -814,22 +814,22 @@ private void ReadCore(BinaryReader r, ClientProtocolVersion srvVer) { case 0: FailureHandler = new NoOpFailureHandler(); - + break; case 1: FailureHandler = new StopNodeFailureHandler(); - + break; case 2: FailureHandler = StopNodeOrHaltFailureHandler.Read(r); - + break; - + default: FailureHandler = null; - + break; } } @@ -960,7 +960,7 @@ public string GridName /// Null property values do not override Spring values. /// Value-typed properties are tracked internally: if setter was not called, Spring value won't be overwritten. /// - /// This merging happens on the top level only; e. g. if there are cache configurations defined in Spring + /// This merging happens on the top level only; e. g. if there are cache configurations defined in Spring /// and in .NET, .NET caches will overwrite Spring caches. /// [SuppressMessage("Microsoft.Design", "CA1056:UriPropertiesShouldNotBeStrings")] @@ -991,7 +991,7 @@ public string GridName /// /// List of additional .Net assemblies to load on Ignite start. Each item can be either - /// fully qualified assembly name, path to assembly to DLL or path to a directory when + /// fully qualified assembly name, path to assembly to DLL or path to a directory when /// assemblies reside. /// [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] @@ -1047,7 +1047,7 @@ public bool ClientMode } /// - /// Gets or sets a set of event types () to be recorded by Ignite. + /// Gets or sets a set of event types () to be recorded by Ignite. /// [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] public ICollection IncludedEventTypes { get; set; } @@ -1187,11 +1187,11 @@ public TimeSpan NetworkTimeout public string WorkDirectory { get; set; } /// - /// Gets or sets system-wide local address or host for all Ignite components to bind to. + /// Gets or sets system-wide local address or host for all Ignite components to bind to. /// If provided it will override all default local bind settings within Ignite. /// - /// If null then Ignite tries to use local wildcard address.That means that all services - /// will be available on all network interfaces of the host machine. + /// If null then Ignite tries to use local wildcard address.That means that all services + /// will be available on all network interfaces of the host machine. /// /// It is strongly recommended to set this parameter for all production environments. /// @@ -1200,11 +1200,11 @@ public TimeSpan NetworkTimeout /// /// Gets or sets a value indicating whether this node should be a daemon node. /// - /// Daemon nodes are the usual grid nodes that participate in topology but not visible on the main APIs, + /// Daemon nodes are the usual grid nodes that participate in topology but not visible on the main APIs, /// i.e. they are not part of any cluster groups. /// - /// Daemon nodes are used primarily for management and monitoring functionality that is built on Ignite - /// and needs to participate in the topology, but also needs to be excluded from the "normal" topology, + /// Daemon nodes are used primarily for management and monitoring functionality that is built on Ignite + /// and needs to participate in the topology, but also needs to be excluded from the "normal" topology, /// so that it won't participate in the task execution or in-memory data grid storage. /// public bool IsDaemon @@ -1241,14 +1241,14 @@ public bool IsDaemon /// affinity assignment mode is disabled then new affinity mapping is applied immediately. /// /// With late affinity assignment mode, if primary node was changed for some partition, but data for this - /// partition is not rebalanced yet on this node, then current primary is not changed and new primary - /// is temporary assigned as backup. This nodes becomes primary only when rebalancing for all assigned primary - /// partitions is finished. This mode can show better performance for cache operations, since when cache - /// primary node executes some operation and data is not rebalanced yet, then it sends additional message + /// partition is not rebalanced yet on this node, then current primary is not changed and new primary + /// is temporary assigned as backup. This nodes becomes primary only when rebalancing for all assigned primary + /// partitions is finished. This mode can show better performance for cache operations, since when cache + /// primary node executes some operation and data is not rebalanced yet, then it sends additional message /// to force rebalancing from other nodes. /// /// Note, that interface provides assignment information taking late assignment - /// into account, so while rebalancing for new primary nodes is not finished it can return assignment + /// into account, so while rebalancing for new primary nodes is not finished it can return assignment /// which differs from assignment calculated by AffinityFunction. /// /// This property should have the same value for all nodes in cluster. @@ -1313,7 +1313,7 @@ public static IgniteConfiguration FromXml(string xml) public ILogger Logger { get; set; } /// - /// Gets or sets the failure detection timeout used by + /// Gets or sets the failure detection timeout used by /// and . /// [DefaultValue(typeof(TimeSpan), "00:00:10")] @@ -1540,7 +1540,7 @@ public bool IsActiveOnStart public bool RedirectJavaConsoleOutput { get; set; } /// - /// Gets or sets whether user authentication is enabled for the cluster. Default is false. + /// Gets or sets whether user authentication is enabled for the cluster. Default is false. /// [DefaultValue(DefaultAuthenticationEnabled)] public bool AuthenticationEnabled @@ -1557,7 +1557,7 @@ public bool AuthenticationEnabled /// -- try to stop node if tryStop value is true. /// If node can't be stopped during provided timeout or tryStop value is false then JVM process will be terminated forcibly. /// - /// Only these implementations are supported: + /// Only these implementations are supported: /// , , . ///
    public IFailureHandler FailureHandler { get; set; } @@ -1568,6 +1568,6 @@ public bool AuthenticationEnabled /// /// By default schema names are case-insensitive. Use quotes to enforce case sensitivity. /// - public ICollection SqlSchemas { get; set; } + public ICollection SqlSchemas { get; set; } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs index 0d6306a5eab21..9a684d9b8c33c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs @@ -33,6 +33,9 @@ namespace Apache.Ignite.Linq.Impl /// internal static class ExpressionWalker { + /** SQL quote */ + private const string SqlQuote = "\""; + /** Compiled member readers. */ private static readonly CopyOnWriteConcurrentDictionary> MemberReaders = new CopyOnWriteConcurrentDictionary>(); @@ -251,10 +254,57 @@ public static string GetTableNameWithSchema(ICacheQueryableInternal queryable) var cacheCfg = queryable.CacheConfiguration; - return string.Format(cacheCfg.SqlEscapeAll - ? "\"{0}\".\"{1}\"" - : "\"{0}\".{1}", - cacheCfg.Name, queryable.TableName); + var tableName = queryable.TableName; + if (cacheCfg.SqlEscapeAll) + { + tableName = string.Format("{0}{1}{0}", SqlQuote, tableName); + } + + var schemaName = NormalizeSchemaName(cacheCfg.Name, cacheCfg.SqlSchema); + + return string.Format("{0}.{1}", schemaName, tableName); + } + + /// + /// Normalizes SQL schema name, see + /// org.apache.ignite.internal.processors.query.QueryUtils#normalizeSchemaName + /// + private static string NormalizeSchemaName(string cacheName, string schemaName) + { + if (schemaName == null) + { + // If schema name is not set explicitly, we will use escaped cache name. The reason is that cache name + // could contain weird characters, such as underscores, dots or non-Latin stuff, which are invalid from + // SQL syntax perspective. We do not want node to fail on startup due to this. + return string.Format("{0}{1}{0}", SqlQuote, cacheName); + } + + if (schemaName.StartsWith(SqlQuote, StringComparison.Ordinal) + && schemaName.EndsWith(SqlQuote, StringComparison.Ordinal)) + { + return schemaName; + } + + return NormalizeObjectName(schemaName, false); + } + + /// + /// Normalizes SQL object name, see + /// org.apache.ignite.internal.processors.query.QueryUtils#normalizeObjectName + /// + private static string NormalizeObjectName(string name, bool replace) + { + if (string.IsNullOrEmpty(name)) + { + return name; + } + + if (replace) + { + name = name.Replace('.', '_').Replace('$', '_'); + } + + return name.ToUpperInvariant(); } } } From deb08a99683ff3954dfeeb929e1e163007cc782b Mon Sep 17 00:00:00 2001 From: Oleg Ignatenko Date: Fri, 31 Aug 2018 11:37:10 +0300 Subject: [PATCH 24/95] IGNITE-9348: ML examples improvements, follow-up to IGNITE-9297 this closes #4641 --- .../KMeansClusterizationExample.java | 3 +- .../AlgorithmSpecificDatasetExample.java | 5 +- .../ml/dataset/LocalDatasetExample.java | 69 -- .../change/OptimizeMakeChangeGAExample.java | 14 +- .../OptimizeMakeChangeTerminateCriteria.java | 42 +- .../helloworld/HelloWorldGAExample.java | 13 +- .../HelloWorldTerminateCriteria.java | 34 +- .../genetic/knapsack/KnapsackGAExample.java | 12 +- .../knapsack/KnapsackTerminateCriteria.java | 47 +- .../ml/genetic/movie/MovieGAExample.java | 6 +- .../genetic/movie/MovieTerminateCriteria.java | 42 +- .../ml/knn/ANNClassificationExample.java | 3 +- .../ml/knn/KNNClassificationExample.java | 3 +- .../examples/ml/knn/KNNRegressionExample.java | 3 +- .../examples/ml/nn/MLPTrainerExample.java | 3 +- .../LinearRegressionLSQRTrainerExample.java | 3 +- ...ionLSQRTrainerWithMinMaxScalerExample.java | 3 +- .../LinearRegressionSGDTrainerExample.java | 5 +- .../LogisticRegressionSGDTrainerExample.java | 4 +- .../selection/cv/CrossValidationExample.java | 6 +- .../RandomForestClassificationExample.java | 51 +- .../RandomForestRegressionExample.java | 1064 +++++++++-------- .../ml/environment/EnvironmentTestSuite.java | 1 + .../environment/LearningEnvironmentTest.java | 201 ++++ 24 files changed, 920 insertions(+), 717 deletions(-) delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java index c310302f3d0f9..b96cbce9e55a2 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java @@ -38,7 +38,8 @@ * Code in this example launches Ignite grid and fills the cache with test data points (based on the * Iris dataset).

    *

    - * After that it trains the model based on the specified data using KMeans algorithm.

    + * After that it trains the model based on the specified data using + * KMeans algorithm.

    *

    * Finally, this example loops over the test set of data points, applies the trained model to predict what cluster * does this point belong to, and compares prediction to expected outcome (ground truth).

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java index de7f0e91f925d..4d42d19b1ab41 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/AlgorithmSpecificDatasetExample.java @@ -34,8 +34,9 @@ import org.apache.ignite.ml.math.primitives.vector.VectorUtils; /** - * Example that shows how to implement your own algorithm (gradient descent trainer for linear regression) which uses - * dataset as an underlying infrastructure. + * Example that shows how to implement your own algorithm + * (gradient descent trainer for linear regression) + * which uses dataset as an underlying infrastructure. *

    * Code in this example launches Ignite grid and fills the cache with simple test data.

    *

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java deleted file mode 100644 index 282e76bdce017..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/dataset/LocalDatasetExample.java +++ /dev/null @@ -1,69 +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.examples.ml.dataset; - -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.examples.ml.dataset.model.Person; -import org.apache.ignite.examples.ml.util.DatasetHelper; -import org.apache.ignite.ml.dataset.DatasetFactory; -import org.apache.ignite.ml.dataset.primitive.SimpleDataset; -import org.apache.ignite.ml.math.primitives.vector.VectorUtils; - -/** - * Example that shows how to create dataset based on an existing local storage and then use it to calculate {@code mean} - * and {@code std} values as well as {@code covariance} and {@code correlation} matrices. - *

    - * Code in this example the storage with simple test data.

    - *

    - * After that it creates the dataset based on the data in the storage and uses Dataset API to find and output - * various statistical metrics of the data.

    - *

    - * You can change the test data used in this example and re-run it to explore this functionality further.

    - */ -public class LocalDatasetExample { - /** Run example. */ - public static void main(String[] args) throws Exception { - System.out.println(">>> Local Dataset example started."); - - Map persons = createCache(); - - // Creates a local simple dataset containing features and providing standard dataset API. - try (SimpleDataset dataset = DatasetFactory.createSimpleDataset( - persons, - 2, - (k, v) -> VectorUtils.of(v.getAge(), v.getSalary()) - )) { - new DatasetHelper(dataset).describe(); - } - - System.out.println(">>> Local Dataset example completed."); - } - - /** */ - private static Map createCache() { - Map persons = new HashMap<>(); - - persons.put(1, new Person("Mike", 42, 10000)); - persons.put(2, new Person("John", 32, 64000)); - persons.put(3, new Person("George", 53, 120000)); - persons.put(4, new Person("Karl", 24, 70000)); - - return persons; - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeGAExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeGAExample.java index ed392c29e93e7..36a5f47be9889 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeGAExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeGAExample.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.List; import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.Ignition; import org.apache.ignite.ml.genetic.Chromosome; import org.apache.ignite.ml.genetic.GAGrid; @@ -64,8 +63,6 @@ public class OptimizeMakeChangeGAExample { public static void main(String args[]) { System.out.println(">>> OptimizeMakeChange GA grid example started."); - System.setProperty("IGNITE_QUIET", "false"); - String sAmountChange = "75"; StringBuilder sbErrorMsg = new StringBuilder(); @@ -85,8 +82,6 @@ public static void main(String args[]) { // Create an Ignite instance as you would in any other use case. Ignite ignite = Ignition.start("examples/config/example-ignite.xml"); - IgniteLogger log = ignite.log(); - // Create GAConfiguration. GAConfiguration gaCfg = new GAConfiguration(); @@ -120,7 +115,8 @@ public static void main(String args[]) { gaCfg.setFitnessFunction(function); // Create and set TerminateCriteria. - OptimizeMakeChangeTerminateCriteria termCriteria = new OptimizeMakeChangeTerminateCriteria(ignite); + OptimizeMakeChangeTerminateCriteria termCriteria = new OptimizeMakeChangeTerminateCriteria(ignite, + System.out::println); ChromosomeCriteria chromosomeCriteria = new ChromosomeCriteria(); @@ -139,11 +135,11 @@ public static void main(String args[]) { // Initialize GAGrid. GAGrid gaGrid = new GAGrid(gaCfg, ignite); - log.info("##########################################################################################"); + System.out.println("##########################################################################################"); - log.info("Calculating optimal set of coins where amount of change is " + sAmountChange); + System.out.println("Calculating optimal set of coins where amount of change is " + sAmountChange); - log.info("##########################################################################################"); + System.out.println("##########################################################################################"); Chromosome chromosome = gaGrid.evolve(); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeTerminateCriteria.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeTerminateCriteria.java index 1d6612c28e793..2d65d3f3c600f 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeTerminateCriteria.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeTerminateCriteria.java @@ -18,8 +18,8 @@ package org.apache.ignite.examples.ml.genetic.change; import java.util.List; +import java.util.function.Consumer; import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.ml.genetic.Chromosome; import org.apache.ignite.ml.genetic.Gene; import org.apache.ignite.ml.genetic.parameter.ITerminateCriteria; @@ -29,19 +29,21 @@ * Terminate Condition implementation for {@link OptimizeMakeChangeGAExample}. */ public class OptimizeMakeChangeTerminateCriteria implements ITerminateCriteria { - /** Ignite logger. */ - private IgniteLogger igniteLog; - /** Ignite instance. */ - private Ignite ignite; + /** */ + private final Ignite ignite; + + /** */ + private final Consumer logConsumer; /** * Create class instance. * * @param ignite Ignite instance. + * @param logConsumer Logging consumer. */ - public OptimizeMakeChangeTerminateCriteria(Ignite ignite) { + OptimizeMakeChangeTerminateCriteria(Ignite ignite, Consumer logConsumer) { this.ignite = ignite; - this.igniteLog = ignite.log(); + this.logConsumer = logConsumer; } /** @@ -56,13 +58,14 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av int currGeneration) { boolean isTerminate = true; - igniteLog.info("##########################################################################################"); - igniteLog.info("Generation: " + currGeneration); - igniteLog.info("Fittest is Chromosome Key: " + fittestChromosome); - igniteLog.info("Chromosome: " + fittestChromosome); - printCoins(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)); - igniteLog.info("Avg Chromosome Fitness: " + averageFitnessScore); - igniteLog.info("##########################################################################################"); + logConsumer.accept( + "\n##########################################################################################" + + "\n Generation: " + currGeneration + + "\n Fittest is Chromosome Key: " + fittestChromosome + + "\n Chromosome: " + fittestChromosome + + "\n" + reportCoins(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)) + + "\nAvg Chromosome Fitness: " + averageFitnessScore + + "\n##########################################################################################"); if (!(currGeneration > 5)) isTerminate = false; @@ -74,11 +77,16 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av * Helper to print change details. * * @param genes List if Genes. + * @return Details to print. */ - private void printCoins(List genes) { + private String reportCoins(List genes) { + StringBuilder sb = new StringBuilder(); + for (Gene gene : genes) { - igniteLog.info("Coin Type: " + ((Coin)gene.getVal()).getCoinType().toString()); - igniteLog.info("Number of Coins: " + ((Coin)gene.getVal()).getNumOfCoins()); + sb.append("\nCoin Type: ").append(((Coin)gene.getVal()).getCoinType().toString()) + .append("\nNumber of Coins: ").append(((Coin)gene.getVal()).getNumOfCoins()); } + + return sb.toString(); } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldGAExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldGAExample.java index 3182d2f1e8470..585cbb5116e9a 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldGAExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldGAExample.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.Ignition; import org.apache.ignite.ml.genetic.Chromosome; @@ -56,8 +57,6 @@ public class HelloWorldGAExample { public static void main(String args[]) { System.out.println(">>> HelloWorld GA grid example started."); - System.setProperty("IGNITE_QUIET", "false"); - try { // Create an Ignite instance as you would in any other use case. Ignite ignite = Ignition.start("examples/config/example-ignite.xml"); @@ -79,10 +78,14 @@ public static void main(String args[]) { gaCfg.setFitnessFunction(function); // Create and set TerminateCriteria. - HelloWorldTerminateCriteria termCriteria = new HelloWorldTerminateCriteria(ignite); - gaCfg.setTerminateCriteria(termCriteria); + AtomicInteger cnt = new AtomicInteger(0); + HelloWorldTerminateCriteria termCriteria = new HelloWorldTerminateCriteria(ignite, + msg -> { + if (cnt.getAndIncrement() % 20 == 0) + System.out.println(msg); + }); - ignite.log(); + gaCfg.setTerminateCriteria(termCriteria); GAGrid gaGrid = new GAGrid(gaCfg, ignite); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldTerminateCriteria.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldTerminateCriteria.java index f149da12bb551..610d4790a2144 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldTerminateCriteria.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldTerminateCriteria.java @@ -18,8 +18,8 @@ package org.apache.ignite.examples.ml.genetic.helloworld; import java.util.List; +import java.util.function.Consumer; import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.ml.genetic.Chromosome; import org.apache.ignite.ml.genetic.Gene; import org.apache.ignite.ml.genetic.parameter.ITerminateCriteria; @@ -31,19 +31,21 @@ * Class terminates Genetic algorithm when fitness score is more than 10.

    */ public class HelloWorldTerminateCriteria implements ITerminateCriteria { - /** Ignite logger. */ - private IgniteLogger igniteLog; /** Ignite instance. */ - private Ignite ignite; + private final Ignite ignite; + + /** */ + private final Consumer logConsumer; /** * Create class instance. * * @param ignite Ignite instance. + * @param logConsumer Logging consumer. */ - public HelloWorldTerminateCriteria(Ignite ignite) { + HelloWorldTerminateCriteria(Ignite ignite, Consumer logConsumer) { this.ignite = ignite; - this.igniteLog = ignite.log(); + this.logConsumer = logConsumer; } /** @@ -58,13 +60,14 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av int currGeneration) { boolean isTerminate = true; - igniteLog.info("##########################################################################################"); - igniteLog.info("Generation: " + currGeneration); - igniteLog.info("Fittest is Chromosome Key: " + fittestChromosome); - igniteLog.info("Chromosome: " + fittestChromosome); - printPhrase(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)); - igniteLog.info("Avg Chromosome Fitness: " + averageFitnessScore); - igniteLog.info("##########################################################################################"); + logConsumer.accept( + "\n##########################################################################################" + + "\n Generation: " + currGeneration + + "\n Fittest is Chromosome Key: " + fittestChromosome + + "\n Chromosome: " + fittestChromosome + + "\n" + printPhrase(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)) + + "\nAvg Chromosome Fitness: " + averageFitnessScore + + "\n##########################################################################################"); if (!(fittestChromosome.getFitnessScore() > 10)) isTerminate = false; @@ -76,13 +79,14 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av * Helper to print phrase. * * @param genes List of Genes. + * @return Phrase to print. */ - private void printPhrase(List genes) { + private String printPhrase(List genes) { StringBuilder sbPhrase = new StringBuilder(); for (Gene gene : genes) sbPhrase.append(((Character)gene.getVal()).toString()); - igniteLog.info(sbPhrase.toString()); + return sbPhrase.toString(); } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackGAExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackGAExample.java index 873c1c68ebcce..1631a953e927c 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackGAExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackGAExample.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.Ignition; import org.apache.ignite.ml.genetic.Chromosome; @@ -55,8 +56,6 @@ public class KnapsackGAExample { public static void main(String args[]) { System.out.println(">>> Knapsack GA grid example started."); - System.setProperty("IGNITE_QUIET", "false"); - try { // Create an Ignite instance as you would in any other use case. Ignite ignite = Ignition.start("examples/config/example-ignite.xml"); @@ -78,11 +77,14 @@ public static void main(String args[]) { gaCfg.setFitnessFunction(function); // Create and set TerminateCriteria. - KnapsackTerminateCriteria termCriteria = new KnapsackTerminateCriteria(ignite); + AtomicInteger cnt = new AtomicInteger(0); + KnapsackTerminateCriteria termCriteria = new KnapsackTerminateCriteria(ignite, + msg -> { + if (cnt.getAndIncrement() % 10 == 0) + System.out.println(msg); + }); gaCfg.setTerminateCriteria(termCriteria); - ignite.log(); - GAGrid gaGrid = new GAGrid(gaCfg, ignite); // Evolve the population. diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackTerminateCriteria.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackTerminateCriteria.java index 1bba15989422d..09e047ba4f30f 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackTerminateCriteria.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackTerminateCriteria.java @@ -18,8 +18,8 @@ package org.apache.ignite.examples.ml.genetic.knapsack; import java.util.List; +import java.util.function.Consumer; import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.ml.genetic.Chromosome; import org.apache.ignite.ml.genetic.Gene; import org.apache.ignite.ml.genetic.parameter.ITerminateCriteria; @@ -32,19 +32,20 @@ */ public class KnapsackTerminateCriteria implements ITerminateCriteria { /** Ignite instance. */ - private Ignite ignite; + private final Ignite ignite; - /** Ignite logger. */ - private IgniteLogger igniteLog; + /** */ + private final Consumer logConsumer; /** * Create class instance. * * @param ignite Ignite instance. + * @param logConsumer Logging consumer. */ - public KnapsackTerminateCriteria(Ignite ignite) { + KnapsackTerminateCriteria(Ignite ignite, Consumer logConsumer) { this.ignite = ignite; - this.igniteLog = this.ignite.log(); + this.logConsumer = logConsumer; } /** @@ -59,15 +60,16 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av int currGeneration) { boolean isTerminate = true; - igniteLog.info("##########################################################################################"); - igniteLog.info("Generation: " + currGeneration); - igniteLog.info("Fittest is Chromosome Key: " + fittestChromosome); - igniteLog.info("Total value is: " + fittestChromosome.getFitnessScore()); - igniteLog.info("Total weight is: " + calculateTotalWeight(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome))); - igniteLog.info("Avg Chromosome Fitness: " + averageFitnessScore); - igniteLog.info("Chromosome: " + fittestChromosome); - printItems(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)); - igniteLog.info("##########################################################################################"); + logConsumer.accept( + "\n##########################################################################################" + + "\n Generation: " + currGeneration + + "\n Fittest is Chromosome Key: " + fittestChromosome + + "\nTotal value is: " + fittestChromosome.getFitnessScore() + + "\nTotal weight is: " + calculateTotalWeight( + GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)) + + "\nChromosome: " + fittestChromosome + + "\n" + reportItems(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)) + + "\n##########################################################################################"); if (!(currGeneration > 29)) isTerminate = false; @@ -93,13 +95,18 @@ private double calculateTotalWeight(List genes) { * Helper to print items in knapsack. * * @param genes List of Genes. + * @return Items to print. */ - private void printItems(List genes) { + private String reportItems(List genes) { + StringBuilder sb = new StringBuilder(); + for (Gene gene : genes) { - igniteLog.info("------------------------------------------------------------------------------------------"); - igniteLog.info("Name: " + ((Item)gene.getVal()).getName()); - igniteLog.info("Weight: " + ((Item)gene.getVal()).getWeight()); - igniteLog.info("Value: " + ((Item)gene.getVal()).getVal()); + sb.append("\n------------------------------------------------------------------------------------------") + .append("\nName: ").append(((Item)gene.getVal()).getName()) + .append("\nWeight: ").append(((Item)gene.getVal()).getWeight()) + .append("\nValue: ").append(((Item)gene.getVal()).getVal()); } + + return sb.toString(); } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieGAExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieGAExample.java index 51e28beb574ac..cec2719a5eda7 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieGAExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieGAExample.java @@ -61,8 +61,6 @@ public class MovieGAExample { public static void main(String args[]) { System.out.println(">>> Movie GA grid example started."); - System.setProperty("IGNITE_QUIET", "false"); - List genres = new ArrayList<>(); String sGenres = "Action,Comedy,Romance"; @@ -110,14 +108,12 @@ public static void main(String args[]) { // Create an Ignite instance as you would in any other use case. Ignite ignite = Ignition.start("examples/config/example-ignite.xml"); - MovieTerminateCriteria termCriteria = new MovieTerminateCriteria(ignite); + MovieTerminateCriteria termCriteria = new MovieTerminateCriteria(ignite, System.out::println); gaCfg.setTerminateCriteria(termCriteria); GAGrid gaGrid = new GAGrid(gaCfg, ignite); - ignite.log(); - Chromosome chromosome = gaGrid.evolve(); System.out.println(">>> Evolution result: " + chromosome); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieTerminateCriteria.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieTerminateCriteria.java index 7110a7c001447..68c577f4ecbc5 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieTerminateCriteria.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieTerminateCriteria.java @@ -18,8 +18,8 @@ package org.apache.ignite.examples.ml.genetic.movie; import java.util.List; +import java.util.function.Consumer; import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.ml.genetic.Chromosome; import org.apache.ignite.ml.genetic.Gene; import org.apache.ignite.ml.genetic.parameter.ITerminateCriteria; @@ -31,19 +31,21 @@ * Class terminates Genetic algorithm when fitness score is more than 32.

    */ public class MovieTerminateCriteria implements ITerminateCriteria { - /** Ignite logger. */ - private IgniteLogger igniteLog; /** Ignite instance. */ - private Ignite ignite; + private final Ignite ignite; + + /** */ + private final Consumer logConsumer; /** * Create class instance. * * @param ignite Ignite instance. + * @param logConsumer Logging consumer. */ - public MovieTerminateCriteria(Ignite ignite) { + MovieTerminateCriteria(Ignite ignite, Consumer logConsumer) { this.ignite = ignite; - this.igniteLog = ignite.log(); + this.logConsumer = logConsumer; } @@ -59,12 +61,13 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av int currGeneration) { boolean isTerminate = true; - igniteLog.info("##########################################################################################"); - igniteLog.info("Generation: " + currGeneration); - igniteLog.info("Fittest is Chromosome Key: " + fittestChromosome); - igniteLog.info("Chromosome: " + fittestChromosome); - printMovies(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)); - igniteLog.info("##########################################################################################"); + logConsumer.accept( + "\n##########################################################################################" + + "\n Generation: " + currGeneration + + "\n Fittest is Chromosome Key: " + fittestChromosome + + "\nChromosome: " + fittestChromosome + + "\n" + reportMovies(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)) + + "\n##########################################################################################"); if (!(fittestChromosome.getFitnessScore() > 32)) isTerminate = false; @@ -73,15 +76,20 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av } /** - * Helper to print change details. + * Helper to print movies details. * * @param genes List of Genes. + * @return Movies details. */ - private void printMovies(List genes) { + private String reportMovies(List genes) { + StringBuilder sb = new StringBuilder(); + for (Gene gene : genes) { - igniteLog.info("Name: " + ((Movie)gene.getVal()).getName()); - igniteLog.info("Genres: " + ((Movie)gene.getVal()).getGenre().toString()); - igniteLog.info("IMDB Rating: " + ((Movie)gene.getVal()).getImdbRating()); + sb.append("\nName: ").append(((Movie)gene.getVal()).getName()) + .append("\nGenres: ").append(((Movie)gene.getVal()).getGenre().toString()) + .append("\nIMDB Rating: ").append(((Movie)gene.getVal()).getImdbRating()); } + + return sb.toString(); } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/knn/ANNClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/knn/ANNClassificationExample.java index 37cb231dc35d2..8a2d786d479cd 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/knn/ANNClassificationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/knn/ANNClassificationExample.java @@ -42,7 +42,8 @@ * Code in this example launches Ignite grid and fills the cache with test data points (based on the * Iris dataset).

    *

    - * After that it trains the model based on the specified data using kNN algorithm.

    + * After that it trains the model based on the specified data using + * kNN algorithm.

    *

    * Finally, this example loops over the test set of data points, applies the trained model to predict what cluster * does this point belong to, and compares prediction to expected outcome (ground truth).

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java index 77a48ad90a39d..cf285a4358daf 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java @@ -39,7 +39,8 @@ * Code in this example launches Ignite grid and fills the cache with test data points (based on the * Iris dataset).

    *

    - * After that it trains the model based on the specified data using kNN algorithm.

    + * After that it trains the model based on the specified data using + * kNN algorithm.

    *

    * Finally, this example loops over the test set of data points, applies the trained model to predict what cluster * does this point belong to, and compares prediction to expected outcome (ground truth).

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java index a28ecc4a1cae8..78f38c84917b9 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java @@ -39,7 +39,8 @@ * Code in this example launches Ignite grid and fills the cache with test data points (based on the * Iris dataset).

    *

    - * After that it trains the model based on the specified data using kNN regression algorithm.

    + * After that it trains the model based on the specified data using + * kNN regression algorithm.

    *

    * Finally, this example loops over the test set of data points, applies the trained model to predict what cluster * does this point belong to, and compares prediction to expected outcome (ground truth).

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java index 30f876978e569..3e5a98cc4d5ab 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java @@ -41,7 +41,8 @@ *

    * Code in this example launches Ignite grid and fills the cache with simple test data.

    *

    - * After that it defines a layered architecture and a neural network trainer, trains neural network + * After that it defines a layered architecture and a + * neural network trainer, trains neural network * and obtains multilayer perceptron model.

    *

    * Finally, this example loops over the test set, applies the trained model to predict the value and diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java index 085a089937f66..6ac445c4b686f 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java @@ -32,7 +32,8 @@ import org.apache.ignite.thread.IgniteThread; /** - * Run linear regression model ({@link LinearRegressionLSQRTrainer}) over cached dataset. + * Run linear regression model based on LSQR algorithm + * ({@link LinearRegressionLSQRTrainer}) over cached dataset. *

    * Code in this example launches Ignite grid and fills the cache with simple test data.

    *

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithMinMaxScalerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithMinMaxScalerExample.java index 39f3771aa239f..320d4642593b5 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithMinMaxScalerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithMinMaxScalerExample.java @@ -35,7 +35,8 @@ import org.apache.ignite.thread.IgniteThread; /** - * Run linear regression model ({@link LinearRegressionLSQRTrainer}) over cached dataset that was created using + * Run linear regression model based on LSQR algorithm + * ({@link LinearRegressionLSQRTrainer}) over cached dataset that was created using * a minmaxscaling preprocessor ({@link MinMaxScalerTrainer}, {@link MinMaxScalerPreprocessor}). *

    * Code in this example launches Ignite grid, fills the cache with simple test data, and defines minmaxscaling diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java index 0a1e9669a6e12..9fdc0df313877 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java @@ -35,8 +35,9 @@ import org.apache.ignite.thread.IgniteThread; /** - * Run linear regression model based on stochastic gradient descent algorithm ({@link LinearRegressionSGDTrainer}) - * over cached dataset. + * Run linear regression model based on based on + * stochastic gradient descent algorithm + * ({@link LinearRegressionSGDTrainer}) over cached dataset. *

    * Code in this example launches Ignite grid and fills the cache with simple test data.

    *

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java index 7e1c773441d72..0a6ff015cd9c0 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/logistic/binary/LogisticRegressionSGDTrainerExample.java @@ -35,8 +35,8 @@ import org.apache.ignite.thread.IgniteThread; /** - * Run logistic regression model based on stochastic gradient descent algorithm ({@link LogisticRegressionSGDTrainer}) - * over distributed cache. + * Run logistic regression model based on + * stochastic gradient descent algorithm ({@link LogisticRegressionSGDTrainer}) over distributed cache. *

    * Code in this example launches Ignite grid and fills the cache with test data points (based on the * Iris dataset).

    diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/selection/cv/CrossValidationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/selection/cv/CrossValidationExample.java index f1b565052dc71..eb4c8f35d4edc 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/selection/cv/CrossValidationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/selection/cv/CrossValidationExample.java @@ -33,11 +33,13 @@ import org.apache.ignite.thread.IgniteThread; /** - * Run decision tree classification with cross validation ({@link CrossValidation}). + * Run decision tree classification with + * cross validation ({@link CrossValidation}). *

    * Code in this example launches Ignite grid and fills the cache with pseudo random training data points.

    *

    - * After that it creates classification trainer and computes cross-validated metrics based on the training set.

    + * After that it creates classification trainer ({@link DecisionTreeClassificationTrainer}) and computes cross-validated + * metrics based on the training set.

    */ public class CrossValidationExample { /** diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java index 4ea471cbf8c63..aa13943f58f08 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java @@ -32,18 +32,24 @@ import org.apache.ignite.ml.dataset.feature.FeatureMeta; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.tree.randomforest.RandomForestClassifierTrainer; -import org.apache.ignite.ml.tree.randomforest.RandomForestTrainer; import org.apache.ignite.ml.tree.randomforest.data.FeaturesCountSelectionStrategies; import org.apache.ignite.thread.IgniteThread; /** - * Example represents a solution for the task of wine classification based on RandomForestTrainer implementation for - * multi-classification. It shows an initialization of {@link RandomForestTrainer} with thread pool for multi-thread - * learning, initialization of Ignite Cache, learning step and evaluation of accuracy of model. - * - * Dataset url: https://archive.ics.uci.edu/ml/machine-learning-databases/wine/ - * - * @see RandomForestClassifierTrainer + * Example represents a solution for the task of wine classification based on a + * Random Forest implementation for + * multi-classification. + *

    + * Code in this example launches Ignite grid and fills the cache with test data points (based on the + * Wine recognition dataset).

    + *

    + * After that it initializes the {@link RandomForestClassifierTrainer} with thread pool for multi-thread learning + * and trains the model based on the specified data using random forest regression algorithm.

    + *

    + * Finally, this example loops over the test set of data points, compares prediction of the trained model to the + * expected outcome (ground truth), and evaluates accuracy of the model.

    + *

    + * You can change the test data used in this example and re-run it to explore this algorithm further.

    */ public class RandomForestClassificationExample { /** @@ -57,23 +63,29 @@ public static void main(String[] args) throws InterruptedException { System.out.println(">>> Ignite grid started."); IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - RandomForestClassificationExample.class.getSimpleName(), () -> { + RandomForestClassificationExample.class.getSimpleName(), () -> { IgniteCache dataCache = new TestCache(ignite).fillCacheWith(data); - AtomicInteger indx = new AtomicInteger(0); + AtomicInteger idx = new AtomicInteger(0); RandomForestClassifierTrainer classifier = new RandomForestClassifierTrainer( - IntStream.range(0, 13).mapToObj(x -> new FeatureMeta("", indx.getAndIncrement(), false)).collect(Collectors.toList())) - .withCountOfTrees(101) - .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.SQRT) - .withMaxDepth(3) + IntStream.range(0, data[0].length - 1).mapToObj( + x -> new FeatureMeta("", idx.getAndIncrement(), false)).collect(Collectors.toList()) + ).withCountOfTrees(101) + .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) + .withMaxDepth(4) .withMinImpurityDelta(0.) - .withSubsampleSize(0.3); + .withSubsampleSize(0.3) + .withSeed(0); + + System.out.println(">>> Configured trainer: " + classifier.getClass().getSimpleName()); ModelsComposition randomForest = classifier.fit(ignite, dataCache, (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), (k, v) -> v[0] ); + System.out.println(">>> Trained model: " + randomForest.toString(true)); + int amountOfErrors = 0; int totalAmount = 0; @@ -91,8 +103,11 @@ public static void main(String[] args) throws InterruptedException { } + System.out.println("\n>>> Evaluated model on " + totalAmount + " data points."); + System.out.println("\n>>> Absolute amount of errors " + amountOfErrors); - System.out.println("\n>>> Accuracy " + (1 - amountOfErrors / (double)totalAmount)); + System.out.println("\n>>> Accuracy " + (1 - amountOfErrors / (double) totalAmount)); + System.out.println(">>> Random Forest multi-class classification algorithm over cached dataset usage example completed."); } }); @@ -101,9 +116,7 @@ public static void main(String[] args) throws InterruptedException { } } - /** - * The Wine dataset. - */ + /** The Wine recognition dataset. */ private static final double[][] data = { {1, 14.23, 1.71, 2.43, 15.6, 127, 2.8, 3.06, .28, 2.29, 5.64, 1.04, 3.92, 1065}, {1, 13.2, 1.78, 2.14, 11.2, 100, 2.65, 2.76, .26, 1.28, 4.38, 1.05, 3.4, 1050}, diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java index 96a87c2ebbe03..e2bfe8b84b876 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java @@ -30,21 +30,30 @@ import org.apache.ignite.examples.ml.util.TestCache; import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.environment.LearningEnvironment; +import org.apache.ignite.ml.environment.logging.ConsoleLogger; +import org.apache.ignite.ml.environment.logging.MLLogger; +import org.apache.ignite.ml.environment.parallelism.ParallelismStrategy; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.tree.randomforest.RandomForestRegressionTrainer; -import org.apache.ignite.ml.tree.randomforest.RandomForestTrainer; import org.apache.ignite.ml.tree.randomforest.data.FeaturesCountSelectionStrategies; import org.apache.ignite.thread.IgniteThread; /** - * Example represents a solution for the task of price predictions for houses in Boston based on RandomForestTrainer - * implementation for regression. It shows an initialization of {@link RandomForestTrainer}, +initialization of Ignite - * Cache, learning step and evaluation of model quality in terms of Mean Squared Error (MSE) and Mean Absolute Error - * (MAE). - * - * Dataset url: https://archive.ics.uci.edu/ml/machine-learning-databases/housing/ - * - * @see RandomForestRegressionTrainer + * Example represents a solution for the task of price predictions for houses in Boston based on a + * Random Forest implementation for regression. + *

    + * Code in this example launches Ignite grid and fills the cache with test data points (based on the + * Boston Housing dataset).

    + *

    + * After that it initializes the {@link RandomForestRegressionTrainer} and trains the model based on the specified data + * using random forest regression algorithm.

    + *

    + * Finally, this example loops over the test set of data points, compares prediction of the trained model to the + * expected outcome (ground truth), and evaluates model quality in terms of Mean Squared Error (MSE) and + * Mean Absolute Error (MAE).

    + *

    + * You can change the test data used in this example and re-run it to explore this algorithm further.

    */ public class RandomForestRegressionExample { /** @@ -61,9 +70,10 @@ public static void main(String[] args) throws InterruptedException { RandomForestRegressionExample.class.getSimpleName(), () -> { IgniteCache dataCache = new TestCache(ignite).fillCacheWith(data); - AtomicInteger indx = new AtomicInteger(0); + AtomicInteger idx = new AtomicInteger(0); RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer( - IntStream.range(0, data[0].length - 1).mapToObj(x -> new FeatureMeta("", indx.getAndIncrement(), false)).collect(Collectors.toList()) + IntStream.range(0, data[0].length - 1).mapToObj( + x -> new FeatureMeta("", idx.getAndIncrement(), false)).collect(Collectors.toList()) ).withCountOfTrees(101) .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) .withMaxDepth(4) @@ -71,11 +81,21 @@ public static void main(String[] args) throws InterruptedException { .withSubsampleSize(0.3) .withSeed(0); + trainer.setEnvironment(LearningEnvironment.builder() + .withParallelismStrategy(ParallelismStrategy.Type.ON_DEFAULT_POOL) + .withLoggingFactory(ConsoleLogger.factory(MLLogger.VerboseLevel.LOW)) + .build() + ); + + System.out.println(">>> Configured trainer: " + trainer.getClass().getSimpleName()); + ModelsComposition randomForest = trainer.fit(ignite, dataCache, - (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), - (k, v) -> v[v.length - 1] + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[v.length - 1] ); + System.out.println(">>> Trained model: " + randomForest.toString(true)); + double mse = 0.0; double mae = 0.0; int totalAmount = 0; @@ -94,11 +114,15 @@ public static void main(String[] args) throws InterruptedException { totalAmount++; } + System.out.println("\n>>> Evaluated model on " + totalAmount + " data points."); + mse = mse / totalAmount; System.out.println("\n>>> Mean squared error (MSE) " + mse); mae = mae / totalAmount; System.out.println("\n>>> Mean absolute error (MAE) " + mae); + + System.out.println(">>> Random Forest regression algorithm over cached dataset usage example completed."); } }); @@ -107,514 +131,512 @@ public static void main(String[] args) throws InterruptedException { } } - /** - * The Boston housing dataset. - */ + /** The Boston housing dataset. */ private static final double[][] data = { - {0.02731,0.00,7.070,0,0.4690,6.4210,78.90,4.9671,2,242.0,17.80,396.90,9.14,21.60}, - {0.02729,0.00,7.070,0,0.4690,7.1850,61.10,4.9671,2,242.0,17.80,392.83,4.03,34.70}, - {0.03237,0.00,2.180,0,0.4580,6.9980,45.80,6.0622,3,222.0,18.70,394.63,2.94,33.40}, - {0.06905,0.00,2.180,0,0.4580,7.1470,54.20,6.0622,3,222.0,18.70,396.90,5.33,36.20}, - {0.02985,0.00,2.180,0,0.4580,6.4300,58.70,6.0622,3,222.0,18.70,394.12,5.21,28.70}, - {0.08829,12.50,7.870,0,0.5240,6.0120,66.60,5.5605,5,311.0,15.20,395.60,12.43,22.90}, - {0.14455,12.50,7.870,0,0.5240,6.1720,96.10,5.9505,5,311.0,15.20,396.90,19.15,27.10}, - {0.21124,12.50,7.870,0,0.5240,5.6310,100.00,6.0821,5,311.0,15.20,386.63,29.93,16.50}, - {0.17004,12.50,7.870,0,0.5240,6.0040,85.90,6.5921,5,311.0,15.20,386.71,17.10,18.90}, - {0.22489,12.50,7.870,0,0.5240,6.3770,94.30,6.3467,5,311.0,15.20,392.52,20.45,15.00}, - {0.11747,12.50,7.870,0,0.5240,6.0090,82.90,6.2267,5,311.0,15.20,396.90,13.27,18.90}, - {0.09378,12.50,7.870,0,0.5240,5.8890,39.00,5.4509,5,311.0,15.20,390.50,15.71,21.70}, - {0.62976,0.00,8.140,0,0.5380,5.9490,61.80,4.7075,4,307.0,21.00,396.90,8.26,20.40}, - {0.63796,0.00,8.140,0,0.5380,6.0960,84.50,4.4619,4,307.0,21.00,380.02,10.26,18.20}, - {0.62739,0.00,8.140,0,0.5380,5.8340,56.50,4.4986,4,307.0,21.00,395.62,8.47,19.90}, - {1.05393,0.00,8.140,0,0.5380,5.9350,29.30,4.4986,4,307.0,21.00,386.85,6.58,23.10}, - {0.78420,0.00,8.140,0,0.5380,5.9900,81.70,4.2579,4,307.0,21.00,386.75,14.67,17.50}, - {0.80271,0.00,8.140,0,0.5380,5.4560,36.60,3.7965,4,307.0,21.00,288.99,11.69,20.20}, - {0.72580,0.00,8.140,0,0.5380,5.7270,69.50,3.7965,4,307.0,21.00,390.95,11.28,18.20}, - {1.25179,0.00,8.140,0,0.5380,5.5700,98.10,3.7979,4,307.0,21.00,376.57,21.02,13.60}, - {0.85204,0.00,8.140,0,0.5380,5.9650,89.20,4.0123,4,307.0,21.00,392.53,13.83,19.60}, - {1.23247,0.00,8.140,0,0.5380,6.1420,91.70,3.9769,4,307.0,21.00,396.90,18.72,15.20}, - {0.98843,0.00,8.140,0,0.5380,5.8130,100.00,4.0952,4,307.0,21.00,394.54,19.88,14.50}, - {0.75026,0.00,8.140,0,0.5380,5.9240,94.10,4.3996,4,307.0,21.00,394.33,16.30,15.60}, - {0.84054,0.00,8.140,0,0.5380,5.5990,85.70,4.4546,4,307.0,21.00,303.42,16.51,13.90}, - {0.67191,0.00,8.140,0,0.5380,5.8130,90.30,4.6820,4,307.0,21.00,376.88,14.81,16.60}, - {0.95577,0.00,8.140,0,0.5380,6.0470,88.80,4.4534,4,307.0,21.00,306.38,17.28,14.80}, - {0.77299,0.00,8.140,0,0.5380,6.4950,94.40,4.4547,4,307.0,21.00,387.94,12.80,18.40}, - {1.00245,0.00,8.140,0,0.5380,6.6740,87.30,4.2390,4,307.0,21.00,380.23,11.98,21.00}, - {1.13081,0.00,8.140,0,0.5380,5.7130,94.10,4.2330,4,307.0,21.00,360.17,22.60,12.70}, - {1.35472,0.00,8.140,0,0.5380,6.0720,100.00,4.1750,4,307.0,21.00,376.73,13.04,14.50}, - {1.38799,0.00,8.140,0,0.5380,5.9500,82.00,3.9900,4,307.0,21.00,232.60,27.71,13.20}, - {1.15172,0.00,8.140,0,0.5380,5.7010,95.00,3.7872,4,307.0,21.00,358.77,18.35,13.10}, - {1.61282,0.00,8.140,0,0.5380,6.0960,96.90,3.7598,4,307.0,21.00,248.31,20.34,13.50}, - {0.06417,0.00,5.960,0,0.4990,5.9330,68.20,3.3603,5,279.0,19.20,396.90,9.68,18.90}, - {0.09744,0.00,5.960,0,0.4990,5.8410,61.40,3.3779,5,279.0,19.20,377.56,11.41,20.00}, - {0.08014,0.00,5.960,0,0.4990,5.8500,41.50,3.9342,5,279.0,19.20,396.90,8.77,21.00}, - {0.17505,0.00,5.960,0,0.4990,5.9660,30.20,3.8473,5,279.0,19.20,393.43,10.13,24.70}, - {0.02763,75.00,2.950,0,0.4280,6.5950,21.80,5.4011,3,252.0,18.30,395.63,4.32,30.80}, - {0.03359,75.00,2.950,0,0.4280,7.0240,15.80,5.4011,3,252.0,18.30,395.62,1.98,34.90}, - {0.12744,0.00,6.910,0,0.4480,6.7700,2.90,5.7209,3,233.0,17.90,385.41,4.84,26.60}, - {0.14150,0.00,6.910,0,0.4480,6.1690,6.60,5.7209,3,233.0,17.90,383.37,5.81,25.30}, - {0.15936,0.00,6.910,0,0.4480,6.2110,6.50,5.7209,3,233.0,17.90,394.46,7.44,24.70}, - {0.12269,0.00,6.910,0,0.4480,6.0690,40.00,5.7209,3,233.0,17.90,389.39,9.55,21.20}, - {0.17142,0.00,6.910,0,0.4480,5.6820,33.80,5.1004,3,233.0,17.90,396.90,10.21,19.30}, - {0.18836,0.00,6.910,0,0.4480,5.7860,33.30,5.1004,3,233.0,17.90,396.90,14.15,20.00}, - {0.22927,0.00,6.910,0,0.4480,6.0300,85.50,5.6894,3,233.0,17.90,392.74,18.80,16.60}, - {0.25387,0.00,6.910,0,0.4480,5.3990,95.30,5.8700,3,233.0,17.90,396.90,30.81,14.40}, - {0.21977,0.00,6.910,0,0.4480,5.6020,62.00,6.0877,3,233.0,17.90,396.90,16.20,19.40}, - {0.08873,21.00,5.640,0,0.4390,5.9630,45.70,6.8147,4,243.0,16.80,395.56,13.45,19.70}, - {0.04337,21.00,5.640,0,0.4390,6.1150,63.00,6.8147,4,243.0,16.80,393.97,9.43,20.50}, - {0.05360,21.00,5.640,0,0.4390,6.5110,21.10,6.8147,4,243.0,16.80,396.90,5.28,25.00}, - {0.04981,21.00,5.640,0,0.4390,5.9980,21.40,6.8147,4,243.0,16.80,396.90,8.43,23.40}, - {0.01360,75.00,4.000,0,0.4100,5.8880,47.60,7.3197,3,469.0,21.10,396.90,14.80,18.90}, - {0.01311,90.00,1.220,0,0.4030,7.2490,21.90,8.6966,5,226.0,17.90,395.93,4.81,35.40}, - {0.02055,85.00,0.740,0,0.4100,6.3830,35.70,9.1876,2,313.0,17.30,396.90,5.77,24.70}, - {0.01432,100.00,1.320,0,0.4110,6.8160,40.50,8.3248,5,256.0,15.10,392.90,3.95,31.60}, - {0.15445,25.00,5.130,0,0.4530,6.1450,29.20,7.8148,8,284.0,19.70,390.68,6.86,23.30}, - {0.10328,25.00,5.130,0,0.4530,5.9270,47.20,6.9320,8,284.0,19.70,396.90,9.22,19.60}, - {0.14932,25.00,5.130,0,0.4530,5.7410,66.20,7.2254,8,284.0,19.70,395.11,13.15,18.70}, - {0.17171,25.00,5.130,0,0.4530,5.9660,93.40,6.8185,8,284.0,19.70,378.08,14.44,16.00}, - {0.11027,25.00,5.130,0,0.4530,6.4560,67.80,7.2255,8,284.0,19.70,396.90,6.73,22.20}, - {0.12650,25.00,5.130,0,0.4530,6.7620,43.40,7.9809,8,284.0,19.70,395.58,9.50,25.00}, - {0.01951,17.50,1.380,0,0.4161,7.1040,59.50,9.2229,3,216.0,18.60,393.24,8.05,33.00}, - {0.03584,80.00,3.370,0,0.3980,6.2900,17.80,6.6115,4,337.0,16.10,396.90,4.67,23.50}, - {0.04379,80.00,3.370,0,0.3980,5.7870,31.10,6.6115,4,337.0,16.10,396.90,10.24,19.40}, - {0.05789,12.50,6.070,0,0.4090,5.8780,21.40,6.4980,4,345.0,18.90,396.21,8.10,22.00}, - {0.13554,12.50,6.070,0,0.4090,5.5940,36.80,6.4980,4,345.0,18.90,396.90,13.09,17.40}, - {0.12816,12.50,6.070,0,0.4090,5.8850,33.00,6.4980,4,345.0,18.90,396.90,8.79,20.90}, - {0.08826,0.00,10.810,0,0.4130,6.4170,6.60,5.2873,4,305.0,19.20,383.73,6.72,24.20}, - {0.15876,0.00,10.810,0,0.4130,5.9610,17.50,5.2873,4,305.0,19.20,376.94,9.88,21.70}, - {0.09164,0.00,10.810,0,0.4130,6.0650,7.80,5.2873,4,305.0,19.20,390.91,5.52,22.80}, - {0.19539,0.00,10.810,0,0.4130,6.2450,6.20,5.2873,4,305.0,19.20,377.17,7.54,23.40}, - {0.07896,0.00,12.830,0,0.4370,6.2730,6.00,4.2515,5,398.0,18.70,394.92,6.78,24.10}, - {0.09512,0.00,12.830,0,0.4370,6.2860,45.00,4.5026,5,398.0,18.70,383.23,8.94,21.40}, - {0.10153,0.00,12.830,0,0.4370,6.2790,74.50,4.0522,5,398.0,18.70,373.66,11.97,20.00}, - {0.08707,0.00,12.830,0,0.4370,6.1400,45.80,4.0905,5,398.0,18.70,386.96,10.27,20.80}, - {0.05646,0.00,12.830,0,0.4370,6.2320,53.70,5.0141,5,398.0,18.70,386.40,12.34,21.20}, - {0.08387,0.00,12.830,0,0.4370,5.8740,36.60,4.5026,5,398.0,18.70,396.06,9.10,20.30}, - {0.04113,25.00,4.860,0,0.4260,6.7270,33.50,5.4007,4,281.0,19.00,396.90,5.29,28.00}, - {0.04462,25.00,4.860,0,0.4260,6.6190,70.40,5.4007,4,281.0,19.00,395.63,7.22,23.90}, - {0.03659,25.00,4.860,0,0.4260,6.3020,32.20,5.4007,4,281.0,19.00,396.90,6.72,24.80}, - {0.03551,25.00,4.860,0,0.4260,6.1670,46.70,5.4007,4,281.0,19.00,390.64,7.51,22.90}, - {0.05059,0.00,4.490,0,0.4490,6.3890,48.00,4.7794,3,247.0,18.50,396.90,9.62,23.90}, - {0.05735,0.00,4.490,0,0.4490,6.6300,56.10,4.4377,3,247.0,18.50,392.30,6.53,26.60}, - {0.05188,0.00,4.490,0,0.4490,6.0150,45.10,4.4272,3,247.0,18.50,395.99,12.86,22.50}, - {0.07151,0.00,4.490,0,0.4490,6.1210,56.80,3.7476,3,247.0,18.50,395.15,8.44,22.20}, - {0.05660,0.00,3.410,0,0.4890,7.0070,86.30,3.4217,2,270.0,17.80,396.90,5.50,23.60}, - {0.05302,0.00,3.410,0,0.4890,7.0790,63.10,3.4145,2,270.0,17.80,396.06,5.70,28.70}, - {0.04684,0.00,3.410,0,0.4890,6.4170,66.10,3.0923,2,270.0,17.80,392.18,8.81,22.60}, - {0.03932,0.00,3.410,0,0.4890,6.4050,73.90,3.0921,2,270.0,17.80,393.55,8.20,22.00}, - {0.04203,28.00,15.040,0,0.4640,6.4420,53.60,3.6659,4,270.0,18.20,395.01,8.16,22.90}, - {0.02875,28.00,15.040,0,0.4640,6.2110,28.90,3.6659,4,270.0,18.20,396.33,6.21,25.00}, - {0.04294,28.00,15.040,0,0.4640,6.2490,77.30,3.6150,4,270.0,18.20,396.90,10.59,20.60}, - {0.12204,0.00,2.890,0,0.4450,6.6250,57.80,3.4952,2,276.0,18.00,357.98,6.65,28.40}, - {0.11504,0.00,2.890,0,0.4450,6.1630,69.60,3.4952,2,276.0,18.00,391.83,11.34,21.40}, - {0.12083,0.00,2.890,0,0.4450,8.0690,76.00,3.4952,2,276.0,18.00,396.90,4.21,38.70}, - {0.08187,0.00,2.890,0,0.4450,7.8200,36.90,3.4952,2,276.0,18.00,393.53,3.57,43.80}, - {0.06860,0.00,2.890,0,0.4450,7.4160,62.50,3.4952,2,276.0,18.00,396.90,6.19,33.20}, - {0.14866,0.00,8.560,0,0.5200,6.7270,79.90,2.7778,5,384.0,20.90,394.76,9.42,27.50}, - {0.11432,0.00,8.560,0,0.5200,6.7810,71.30,2.8561,5,384.0,20.90,395.58,7.67,26.50}, - {0.22876,0.00,8.560,0,0.5200,6.4050,85.40,2.7147,5,384.0,20.90,70.80,10.63,18.60}, - {0.21161,0.00,8.560,0,0.5200,6.1370,87.40,2.7147,5,384.0,20.90,394.47,13.44,19.30}, - {0.13960,0.00,8.560,0,0.5200,6.1670,90.00,2.4210,5,384.0,20.90,392.69,12.33,20.10}, - {0.13262,0.00,8.560,0,0.5200,5.8510,96.70,2.1069,5,384.0,20.90,394.05,16.47,19.50}, - {0.17120,0.00,8.560,0,0.5200,5.8360,91.90,2.2110,5,384.0,20.90,395.67,18.66,19.50}, - {0.13117,0.00,8.560,0,0.5200,6.1270,85.20,2.1224,5,384.0,20.90,387.69,14.09,20.40}, - {0.12802,0.00,8.560,0,0.5200,6.4740,97.10,2.4329,5,384.0,20.90,395.24,12.27,19.80}, - {0.26363,0.00,8.560,0,0.5200,6.2290,91.20,2.5451,5,384.0,20.90,391.23,15.55,19.40}, - {0.10793,0.00,8.560,0,0.5200,6.1950,54.40,2.7778,5,384.0,20.90,393.49,13.00,21.70}, - {0.10084,0.00,10.010,0,0.5470,6.7150,81.60,2.6775,6,432.0,17.80,395.59,10.16,22.80}, - {0.12329,0.00,10.010,0,0.5470,5.9130,92.90,2.3534,6,432.0,17.80,394.95,16.21,18.80}, - {0.22212,0.00,10.010,0,0.5470,6.0920,95.40,2.5480,6,432.0,17.80,396.90,17.09,18.70}, - {0.14231,0.00,10.010,0,0.5470,6.2540,84.20,2.2565,6,432.0,17.80,388.74,10.45,18.50}, - {0.17134,0.00,10.010,0,0.5470,5.9280,88.20,2.4631,6,432.0,17.80,344.91,15.76,18.30}, - {0.13158,0.00,10.010,0,0.5470,6.1760,72.50,2.7301,6,432.0,17.80,393.30,12.04,21.20}, - {0.15098,0.00,10.010,0,0.5470,6.0210,82.60,2.7474,6,432.0,17.80,394.51,10.30,19.20}, - {0.13058,0.00,10.010,0,0.5470,5.8720,73.10,2.4775,6,432.0,17.80,338.63,15.37,20.40}, - {0.14476,0.00,10.010,0,0.5470,5.7310,65.20,2.7592,6,432.0,17.80,391.50,13.61,19.30}, - {0.06899,0.00,25.650,0,0.5810,5.8700,69.70,2.2577,2,188.0,19.10,389.15,14.37,22.00}, - {0.07165,0.00,25.650,0,0.5810,6.0040,84.10,2.1974,2,188.0,19.10,377.67,14.27,20.30}, - {0.09299,0.00,25.650,0,0.5810,5.9610,92.90,2.0869,2,188.0,19.10,378.09,17.93,20.50}, - {0.15038,0.00,25.650,0,0.5810,5.8560,97.00,1.9444,2,188.0,19.10,370.31,25.41,17.30}, - {0.09849,0.00,25.650,0,0.5810,5.8790,95.80,2.0063,2,188.0,19.10,379.38,17.58,18.80}, - {0.16902,0.00,25.650,0,0.5810,5.9860,88.40,1.9929,2,188.0,19.10,385.02,14.81,21.40}, - {0.38735,0.00,25.650,0,0.5810,5.6130,95.60,1.7572,2,188.0,19.10,359.29,27.26,15.70}, - {0.25915,0.00,21.890,0,0.6240,5.6930,96.00,1.7883,4,437.0,21.20,392.11,17.19,16.20}, - {0.32543,0.00,21.890,0,0.6240,6.4310,98.80,1.8125,4,437.0,21.20,396.90,15.39,18.00}, - {0.88125,0.00,21.890,0,0.6240,5.6370,94.70,1.9799,4,437.0,21.20,396.90,18.34,14.30}, - {0.34006,0.00,21.890,0,0.6240,6.4580,98.90,2.1185,4,437.0,21.20,395.04,12.60,19.20}, - {1.19294,0.00,21.890,0,0.6240,6.3260,97.70,2.2710,4,437.0,21.20,396.90,12.26,19.60}, - {0.59005,0.00,21.890,0,0.6240,6.3720,97.90,2.3274,4,437.0,21.20,385.76,11.12,23.00}, - {0.32982,0.00,21.890,0,0.6240,5.8220,95.40,2.4699,4,437.0,21.20,388.69,15.03,18.40}, - {0.97617,0.00,21.890,0,0.6240,5.7570,98.40,2.3460,4,437.0,21.20,262.76,17.31,15.60}, - {0.55778,0.00,21.890,0,0.6240,6.3350,98.20,2.1107,4,437.0,21.20,394.67,16.96,18.10}, - {0.32264,0.00,21.890,0,0.6240,5.9420,93.50,1.9669,4,437.0,21.20,378.25,16.90,17.40}, - {0.35233,0.00,21.890,0,0.6240,6.4540,98.40,1.8498,4,437.0,21.20,394.08,14.59,17.10}, - {0.24980,0.00,21.890,0,0.6240,5.8570,98.20,1.6686,4,437.0,21.20,392.04,21.32,13.30}, - {0.54452,0.00,21.890,0,0.6240,6.1510,97.90,1.6687,4,437.0,21.20,396.90,18.46,17.80}, - {0.29090,0.00,21.890,0,0.6240,6.1740,93.60,1.6119,4,437.0,21.20,388.08,24.16,14.00}, - {1.62864,0.00,21.890,0,0.6240,5.0190,100.00,1.4394,4,437.0,21.20,396.90,34.41,14.40}, - {3.32105,0.00,19.580,1,0.8710,5.4030,100.00,1.3216,5,403.0,14.70,396.90,26.82,13.40}, - {4.09740,0.00,19.580,0,0.8710,5.4680,100.00,1.4118,5,403.0,14.70,396.90,26.42,15.60}, - {2.77974,0.00,19.580,0,0.8710,4.9030,97.80,1.3459,5,403.0,14.70,396.90,29.29,11.80}, - {2.37934,0.00,19.580,0,0.8710,6.1300,100.00,1.4191,5,403.0,14.70,172.91,27.80,13.80}, - {2.15505,0.00,19.580,0,0.8710,5.6280,100.00,1.5166,5,403.0,14.70,169.27,16.65,15.60}, - {2.36862,0.00,19.580,0,0.8710,4.9260,95.70,1.4608,5,403.0,14.70,391.71,29.53,14.60}, - {2.33099,0.00,19.580,0,0.8710,5.1860,93.80,1.5296,5,403.0,14.70,356.99,28.32,17.80}, - {2.73397,0.00,19.580,0,0.8710,5.5970,94.90,1.5257,5,403.0,14.70,351.85,21.45,15.40}, - {1.65660,0.00,19.580,0,0.8710,6.1220,97.30,1.6180,5,403.0,14.70,372.80,14.10,21.50}, - {1.49632,0.00,19.580,0,0.8710,5.4040,100.00,1.5916,5,403.0,14.70,341.60,13.28,19.60}, - {1.12658,0.00,19.580,1,0.8710,5.0120,88.00,1.6102,5,403.0,14.70,343.28,12.12,15.30}, - {2.14918,0.00,19.580,0,0.8710,5.7090,98.50,1.6232,5,403.0,14.70,261.95,15.79,19.40}, - {1.41385,0.00,19.580,1,0.8710,6.1290,96.00,1.7494,5,403.0,14.70,321.02,15.12,17.00}, - {3.53501,0.00,19.580,1,0.8710,6.1520,82.60,1.7455,5,403.0,14.70,88.01,15.02,15.60}, - {2.44668,0.00,19.580,0,0.8710,5.2720,94.00,1.7364,5,403.0,14.70,88.63,16.14,13.10}, - {1.22358,0.00,19.580,0,0.6050,6.9430,97.40,1.8773,5,403.0,14.70,363.43,4.59,41.30}, - {1.34284,0.00,19.580,0,0.6050,6.0660,100.00,1.7573,5,403.0,14.70,353.89,6.43,24.30}, - {1.42502,0.00,19.580,0,0.8710,6.5100,100.00,1.7659,5,403.0,14.70,364.31,7.39,23.30}, - {1.27346,0.00,19.580,1,0.6050,6.2500,92.60,1.7984,5,403.0,14.70,338.92,5.50,27.00}, - {1.46336,0.00,19.580,0,0.6050,7.4890,90.80,1.9709,5,403.0,14.70,374.43,1.73,50.00}, - {1.83377,0.00,19.580,1,0.6050,7.8020,98.20,2.0407,5,403.0,14.70,389.61,1.92,50.00}, - {1.51902,0.00,19.580,1,0.6050,8.3750,93.90,2.1620,5,403.0,14.70,388.45,3.32,50.00}, - {2.24236,0.00,19.580,0,0.6050,5.8540,91.80,2.4220,5,403.0,14.70,395.11,11.64,22.70}, - {2.92400,0.00,19.580,0,0.6050,6.1010,93.00,2.2834,5,403.0,14.70,240.16,9.81,25.00}, - {2.01019,0.00,19.580,0,0.6050,7.9290,96.20,2.0459,5,403.0,14.70,369.30,3.70,50.00}, - {1.80028,0.00,19.580,0,0.6050,5.8770,79.20,2.4259,5,403.0,14.70,227.61,12.14,23.80}, - {2.30040,0.00,19.580,0,0.6050,6.3190,96.10,2.1000,5,403.0,14.70,297.09,11.10,23.80}, - {2.44953,0.00,19.580,0,0.6050,6.4020,95.20,2.2625,5,403.0,14.70,330.04,11.32,22.30}, - {1.20742,0.00,19.580,0,0.6050,5.8750,94.60,2.4259,5,403.0,14.70,292.29,14.43,17.40}, - {2.31390,0.00,19.580,0,0.6050,5.8800,97.30,2.3887,5,403.0,14.70,348.13,12.03,19.10}, - {0.13914,0.00,4.050,0,0.5100,5.5720,88.50,2.5961,5,296.0,16.60,396.90,14.69,23.10}, - {0.09178,0.00,4.050,0,0.5100,6.4160,84.10,2.6463,5,296.0,16.60,395.50,9.04,23.60}, - {0.08447,0.00,4.050,0,0.5100,5.8590,68.70,2.7019,5,296.0,16.60,393.23,9.64,22.60}, - {0.06664,0.00,4.050,0,0.5100,6.5460,33.10,3.1323,5,296.0,16.60,390.96,5.33,29.40}, - {0.07022,0.00,4.050,0,0.5100,6.0200,47.20,3.5549,5,296.0,16.60,393.23,10.11,23.20}, - {0.05425,0.00,4.050,0,0.5100,6.3150,73.40,3.3175,5,296.0,16.60,395.60,6.29,24.60}, - {0.06642,0.00,4.050,0,0.5100,6.8600,74.40,2.9153,5,296.0,16.60,391.27,6.92,29.90}, - {0.05780,0.00,2.460,0,0.4880,6.9800,58.40,2.8290,3,193.0,17.80,396.90,5.04,37.20}, - {0.06588,0.00,2.460,0,0.4880,7.7650,83.30,2.7410,3,193.0,17.80,395.56,7.56,39.80}, - {0.06888,0.00,2.460,0,0.4880,6.1440,62.20,2.5979,3,193.0,17.80,396.90,9.45,36.20}, - {0.09103,0.00,2.460,0,0.4880,7.1550,92.20,2.7006,3,193.0,17.80,394.12,4.82,37.90}, - {0.10008,0.00,2.460,0,0.4880,6.5630,95.60,2.8470,3,193.0,17.80,396.90,5.68,32.50}, - {0.08308,0.00,2.460,0,0.4880,5.6040,89.80,2.9879,3,193.0,17.80,391.00,13.98,26.40}, - {0.06047,0.00,2.460,0,0.4880,6.1530,68.80,3.2797,3,193.0,17.80,387.11,13.15,29.60}, - {0.05602,0.00,2.460,0,0.4880,7.8310,53.60,3.1992,3,193.0,17.80,392.63,4.45,50.00}, - {0.07875,45.00,3.440,0,0.4370,6.7820,41.10,3.7886,5,398.0,15.20,393.87,6.68,32.00}, - {0.12579,45.00,3.440,0,0.4370,6.5560,29.10,4.5667,5,398.0,15.20,382.84,4.56,29.80}, - {0.08370,45.00,3.440,0,0.4370,7.1850,38.90,4.5667,5,398.0,15.20,396.90,5.39,34.90}, - {0.09068,45.00,3.440,0,0.4370,6.9510,21.50,6.4798,5,398.0,15.20,377.68,5.10,37.00}, - {0.06911,45.00,3.440,0,0.4370,6.7390,30.80,6.4798,5,398.0,15.20,389.71,4.69,30.50}, - {0.08664,45.00,3.440,0,0.4370,7.1780,26.30,6.4798,5,398.0,15.20,390.49,2.87,36.40}, - {0.02187,60.00,2.930,0,0.4010,6.8000,9.90,6.2196,1,265.0,15.60,393.37,5.03,31.10}, - {0.01439,60.00,2.930,0,0.4010,6.6040,18.80,6.2196,1,265.0,15.60,376.70,4.38,29.10}, - {0.01381,80.00,0.460,0,0.4220,7.8750,32.00,5.6484,4,255.0,14.40,394.23,2.97,50.00}, - {0.04011,80.00,1.520,0,0.4040,7.2870,34.10,7.3090,2,329.0,12.60,396.90,4.08,33.30}, - {0.04666,80.00,1.520,0,0.4040,7.1070,36.60,7.3090,2,329.0,12.60,354.31,8.61,30.30}, - {0.03768,80.00,1.520,0,0.4040,7.2740,38.30,7.3090,2,329.0,12.60,392.20,6.62,34.60}, - {0.03150,95.00,1.470,0,0.4030,6.9750,15.30,7.6534,3,402.0,17.00,396.90,4.56,34.90}, - {0.01778,95.00,1.470,0,0.4030,7.1350,13.90,7.6534,3,402.0,17.00,384.30,4.45,32.90}, - {0.03445,82.50,2.030,0,0.4150,6.1620,38.40,6.2700,2,348.0,14.70,393.77,7.43,24.10}, - {0.02177,82.50,2.030,0,0.4150,7.6100,15.70,6.2700,2,348.0,14.70,395.38,3.11,42.30}, - {0.03510,95.00,2.680,0,0.4161,7.8530,33.20,5.1180,4,224.0,14.70,392.78,3.81,48.50}, - {0.02009,95.00,2.680,0,0.4161,8.0340,31.90,5.1180,4,224.0,14.70,390.55,2.88,50.00}, - {0.13642,0.00,10.590,0,0.4890,5.8910,22.30,3.9454,4,277.0,18.60,396.90,10.87,22.60}, - {0.22969,0.00,10.590,0,0.4890,6.3260,52.50,4.3549,4,277.0,18.60,394.87,10.97,24.40}, - {0.25199,0.00,10.590,0,0.4890,5.7830,72.70,4.3549,4,277.0,18.60,389.43,18.06,22.50}, - {0.13587,0.00,10.590,1,0.4890,6.0640,59.10,4.2392,4,277.0,18.60,381.32,14.66,24.40}, - {0.43571,0.00,10.590,1,0.4890,5.3440,100.00,3.8750,4,277.0,18.60,396.90,23.09,20.00}, - {0.17446,0.00,10.590,1,0.4890,5.9600,92.10,3.8771,4,277.0,18.60,393.25,17.27,21.70}, - {0.37578,0.00,10.590,1,0.4890,5.4040,88.60,3.6650,4,277.0,18.60,395.24,23.98,19.30}, - {0.21719,0.00,10.590,1,0.4890,5.8070,53.80,3.6526,4,277.0,18.60,390.94,16.03,22.40}, - {0.14052,0.00,10.590,0,0.4890,6.3750,32.30,3.9454,4,277.0,18.60,385.81,9.38,28.10}, - {0.28955,0.00,10.590,0,0.4890,5.4120,9.80,3.5875,4,277.0,18.60,348.93,29.55,23.70}, - {0.19802,0.00,10.590,0,0.4890,6.1820,42.40,3.9454,4,277.0,18.60,393.63,9.47,25.00}, - {0.04560,0.00,13.890,1,0.5500,5.8880,56.00,3.1121,5,276.0,16.40,392.80,13.51,23.30}, - {0.07013,0.00,13.890,0,0.5500,6.6420,85.10,3.4211,5,276.0,16.40,392.78,9.69,28.70}, - {0.11069,0.00,13.890,1,0.5500,5.9510,93.80,2.8893,5,276.0,16.40,396.90,17.92,21.50}, - {0.11425,0.00,13.890,1,0.5500,6.3730,92.40,3.3633,5,276.0,16.40,393.74,10.50,23.00}, - {0.35809,0.00,6.200,1,0.5070,6.9510,88.50,2.8617,8,307.0,17.40,391.70,9.71,26.70}, - {0.40771,0.00,6.200,1,0.5070,6.1640,91.30,3.0480,8,307.0,17.40,395.24,21.46,21.70}, - {0.62356,0.00,6.200,1,0.5070,6.8790,77.70,3.2721,8,307.0,17.40,390.39,9.93,27.50}, - {0.61470,0.00,6.200,0,0.5070,6.6180,80.80,3.2721,8,307.0,17.40,396.90,7.60,30.10}, - {0.31533,0.00,6.200,0,0.5040,8.2660,78.30,2.8944,8,307.0,17.40,385.05,4.14,44.80}, - {0.52693,0.00,6.200,0,0.5040,8.7250,83.00,2.8944,8,307.0,17.40,382.00,4.63,50.00}, - {0.38214,0.00,6.200,0,0.5040,8.0400,86.50,3.2157,8,307.0,17.40,387.38,3.13,37.60}, - {0.41238,0.00,6.200,0,0.5040,7.1630,79.90,3.2157,8,307.0,17.40,372.08,6.36,31.60}, - {0.29819,0.00,6.200,0,0.5040,7.6860,17.00,3.3751,8,307.0,17.40,377.51,3.92,46.70}, - {0.44178,0.00,6.200,0,0.5040,6.5520,21.40,3.3751,8,307.0,17.40,380.34,3.76,31.50}, - {0.53700,0.00,6.200,0,0.5040,5.9810,68.10,3.6715,8,307.0,17.40,378.35,11.65,24.30}, - {0.46296,0.00,6.200,0,0.5040,7.4120,76.90,3.6715,8,307.0,17.40,376.14,5.25,31.70}, - {0.57529,0.00,6.200,0,0.5070,8.3370,73.30,3.8384,8,307.0,17.40,385.91,2.47,41.70}, - {0.33147,0.00,6.200,0,0.5070,8.2470,70.40,3.6519,8,307.0,17.40,378.95,3.95,48.30}, - {0.44791,0.00,6.200,1,0.5070,6.7260,66.50,3.6519,8,307.0,17.40,360.20,8.05,29.00}, - {0.33045,0.00,6.200,0,0.5070,6.0860,61.50,3.6519,8,307.0,17.40,376.75,10.88,24.00}, - {0.52058,0.00,6.200,1,0.5070,6.6310,76.50,4.1480,8,307.0,17.40,388.45,9.54,25.10}, - {0.51183,0.00,6.200,0,0.5070,7.3580,71.60,4.1480,8,307.0,17.40,390.07,4.73,31.50}, - {0.08244,30.00,4.930,0,0.4280,6.4810,18.50,6.1899,6,300.0,16.60,379.41,6.36,23.70}, - {0.09252,30.00,4.930,0,0.4280,6.6060,42.20,6.1899,6,300.0,16.60,383.78,7.37,23.30}, - {0.11329,30.00,4.930,0,0.4280,6.8970,54.30,6.3361,6,300.0,16.60,391.25,11.38,22.00}, - {0.10612,30.00,4.930,0,0.4280,6.0950,65.10,6.3361,6,300.0,16.60,394.62,12.40,20.10}, - {0.10290,30.00,4.930,0,0.4280,6.3580,52.90,7.0355,6,300.0,16.60,372.75,11.22,22.20}, - {0.12757,30.00,4.930,0,0.4280,6.3930,7.80,7.0355,6,300.0,16.60,374.71,5.19,23.70}, - {0.20608,22.00,5.860,0,0.4310,5.5930,76.50,7.9549,7,330.0,19.10,372.49,12.50,17.60}, - {0.19133,22.00,5.860,0,0.4310,5.6050,70.20,7.9549,7,330.0,19.10,389.13,18.46,18.50}, - {0.33983,22.00,5.860,0,0.4310,6.1080,34.90,8.0555,7,330.0,19.10,390.18,9.16,24.30}, - {0.19657,22.00,5.860,0,0.4310,6.2260,79.20,8.0555,7,330.0,19.10,376.14,10.15,20.50}, - {0.16439,22.00,5.860,0,0.4310,6.4330,49.10,7.8265,7,330.0,19.10,374.71,9.52,24.50}, - {0.19073,22.00,5.860,0,0.4310,6.7180,17.50,7.8265,7,330.0,19.10,393.74,6.56,26.20}, - {0.14030,22.00,5.860,0,0.4310,6.4870,13.00,7.3967,7,330.0,19.10,396.28,5.90,24.40}, - {0.21409,22.00,5.860,0,0.4310,6.4380,8.90,7.3967,7,330.0,19.10,377.07,3.59,24.80}, - {0.08221,22.00,5.860,0,0.4310,6.9570,6.80,8.9067,7,330.0,19.10,386.09,3.53,29.60}, - {0.36894,22.00,5.860,0,0.4310,8.2590,8.40,8.9067,7,330.0,19.10,396.90,3.54,42.80}, - {0.04819,80.00,3.640,0,0.3920,6.1080,32.00,9.2203,1,315.0,16.40,392.89,6.57,21.90}, - {0.03548,80.00,3.640,0,0.3920,5.8760,19.10,9.2203,1,315.0,16.40,395.18,9.25,20.90}, - {0.01538,90.00,3.750,0,0.3940,7.4540,34.20,6.3361,3,244.0,15.90,386.34,3.11,44.00}, - {0.61154,20.00,3.970,0,0.6470,8.7040,86.90,1.8010,5,264.0,13.00,389.70,5.12,50.00}, - {0.66351,20.00,3.970,0,0.6470,7.3330,100.00,1.8946,5,264.0,13.00,383.29,7.79,36.00}, - {0.65665,20.00,3.970,0,0.6470,6.8420,100.00,2.0107,5,264.0,13.00,391.93,6.90,30.10}, - {0.54011,20.00,3.970,0,0.6470,7.2030,81.80,2.1121,5,264.0,13.00,392.80,9.59,33.80}, - {0.53412,20.00,3.970,0,0.6470,7.5200,89.40,2.1398,5,264.0,13.00,388.37,7.26,43.10}, - {0.52014,20.00,3.970,0,0.6470,8.3980,91.50,2.2885,5,264.0,13.00,386.86,5.91,48.80}, - {0.82526,20.00,3.970,0,0.6470,7.3270,94.50,2.0788,5,264.0,13.00,393.42,11.25,31.00}, - {0.55007,20.00,3.970,0,0.6470,7.2060,91.60,1.9301,5,264.0,13.00,387.89,8.10,36.50}, - {0.76162,20.00,3.970,0,0.6470,5.5600,62.80,1.9865,5,264.0,13.00,392.40,10.45,22.80}, - {0.78570,20.00,3.970,0,0.6470,7.0140,84.60,2.1329,5,264.0,13.00,384.07,14.79,30.70}, - {0.57834,20.00,3.970,0,0.5750,8.2970,67.00,2.4216,5,264.0,13.00,384.54,7.44,50.00}, - {0.54050,20.00,3.970,0,0.5750,7.4700,52.60,2.8720,5,264.0,13.00,390.30,3.16,43.50}, - {0.09065,20.00,6.960,1,0.4640,5.9200,61.50,3.9175,3,223.0,18.60,391.34,13.65,20.70}, - {0.29916,20.00,6.960,0,0.4640,5.8560,42.10,4.4290,3,223.0,18.60,388.65,13.00,21.10}, - {0.16211,20.00,6.960,0,0.4640,6.2400,16.30,4.4290,3,223.0,18.60,396.90,6.59,25.20}, - {0.11460,20.00,6.960,0,0.4640,6.5380,58.70,3.9175,3,223.0,18.60,394.96,7.73,24.40}, - {0.22188,20.00,6.960,1,0.4640,7.6910,51.80,4.3665,3,223.0,18.60,390.77,6.58,35.20}, - {0.05644,40.00,6.410,1,0.4470,6.7580,32.90,4.0776,4,254.0,17.60,396.90,3.53,32.40}, - {0.09604,40.00,6.410,0,0.4470,6.8540,42.80,4.2673,4,254.0,17.60,396.90,2.98,32.00}, - {0.10469,40.00,6.410,1,0.4470,7.2670,49.00,4.7872,4,254.0,17.60,389.25,6.05,33.20}, - {0.06127,40.00,6.410,1,0.4470,6.8260,27.60,4.8628,4,254.0,17.60,393.45,4.16,33.10}, - {0.07978,40.00,6.410,0,0.4470,6.4820,32.10,4.1403,4,254.0,17.60,396.90,7.19,29.10}, - {0.21038,20.00,3.330,0,0.4429,6.8120,32.20,4.1007,5,216.0,14.90,396.90,4.85,35.10}, - {0.03578,20.00,3.330,0,0.4429,7.8200,64.50,4.6947,5,216.0,14.90,387.31,3.76,45.40}, - {0.03705,20.00,3.330,0,0.4429,6.9680,37.20,5.2447,5,216.0,14.90,392.23,4.59,35.40}, - {0.06129,20.00,3.330,1,0.4429,7.6450,49.70,5.2119,5,216.0,14.90,377.07,3.01,46.00}, - {0.01501,90.00,1.210,1,0.4010,7.9230,24.80,5.8850,1,198.0,13.60,395.52,3.16,50.00}, - {0.00906,90.00,2.970,0,0.4000,7.0880,20.80,7.3073,1,285.0,15.30,394.72,7.85,32.20}, - {0.01096,55.00,2.250,0,0.3890,6.4530,31.90,7.3073,1,300.0,15.30,394.72,8.23,22.00}, - {0.01965,80.00,1.760,0,0.3850,6.2300,31.50,9.0892,1,241.0,18.20,341.60,12.93,20.10}, - {0.03871,52.50,5.320,0,0.4050,6.2090,31.30,7.3172,6,293.0,16.60,396.90,7.14,23.20}, - {0.04590,52.50,5.320,0,0.4050,6.3150,45.60,7.3172,6,293.0,16.60,396.90,7.60,22.30}, - {0.04297,52.50,5.320,0,0.4050,6.5650,22.90,7.3172,6,293.0,16.60,371.72,9.51,24.80}, - {0.03502,80.00,4.950,0,0.4110,6.8610,27.90,5.1167,4,245.0,19.20,396.90,3.33,28.50}, - {0.07886,80.00,4.950,0,0.4110,7.1480,27.70,5.1167,4,245.0,19.20,396.90,3.56,37.30}, - {0.03615,80.00,4.950,0,0.4110,6.6300,23.40,5.1167,4,245.0,19.20,396.90,4.70,27.90}, - {0.08265,0.00,13.920,0,0.4370,6.1270,18.40,5.5027,4,289.0,16.00,396.90,8.58,23.90}, - {0.08199,0.00,13.920,0,0.4370,6.0090,42.30,5.5027,4,289.0,16.00,396.90,10.40,21.70}, - {0.12932,0.00,13.920,0,0.4370,6.6780,31.10,5.9604,4,289.0,16.00,396.90,6.27,28.60}, - {0.05372,0.00,13.920,0,0.4370,6.5490,51.00,5.9604,4,289.0,16.00,392.85,7.39,27.10}, - {0.14103,0.00,13.920,0,0.4370,5.7900,58.00,6.3200,4,289.0,16.00,396.90,15.84,20.30}, - {0.06466,70.00,2.240,0,0.4000,6.3450,20.10,7.8278,5,358.0,14.80,368.24,4.97,22.50}, - {0.05561,70.00,2.240,0,0.4000,7.0410,10.00,7.8278,5,358.0,14.80,371.58,4.74,29.00}, - {0.04417,70.00,2.240,0,0.4000,6.8710,47.40,7.8278,5,358.0,14.80,390.86,6.07,24.80}, - {0.03537,34.00,6.090,0,0.4330,6.5900,40.40,5.4917,7,329.0,16.10,395.75,9.50,22.00}, - {0.09266,34.00,6.090,0,0.4330,6.4950,18.40,5.4917,7,329.0,16.10,383.61,8.67,26.40}, - {0.10000,34.00,6.090,0,0.4330,6.9820,17.70,5.4917,7,329.0,16.10,390.43,4.86,33.10}, - {0.05515,33.00,2.180,0,0.4720,7.2360,41.10,4.0220,7,222.0,18.40,393.68,6.93,36.10}, - {0.05479,33.00,2.180,0,0.4720,6.6160,58.10,3.3700,7,222.0,18.40,393.36,8.93,28.40}, - {0.07503,33.00,2.180,0,0.4720,7.4200,71.90,3.0992,7,222.0,18.40,396.90,6.47,33.40}, - {0.04932,33.00,2.180,0,0.4720,6.8490,70.30,3.1827,7,222.0,18.40,396.90,7.53,28.20}, - {0.49298,0.00,9.900,0,0.5440,6.6350,82.50,3.3175,4,304.0,18.40,396.90,4.54,22.80}, - {0.34940,0.00,9.900,0,0.5440,5.9720,76.70,3.1025,4,304.0,18.40,396.24,9.97,20.30}, - {2.63548,0.00,9.900,0,0.5440,4.9730,37.80,2.5194,4,304.0,18.40,350.45,12.64,16.10}, - {0.79041,0.00,9.900,0,0.5440,6.1220,52.80,2.6403,4,304.0,18.40,396.90,5.98,22.10}, - {0.26169,0.00,9.900,0,0.5440,6.0230,90.40,2.8340,4,304.0,18.40,396.30,11.72,19.40}, - {0.26938,0.00,9.900,0,0.5440,6.2660,82.80,3.2628,4,304.0,18.40,393.39,7.90,21.60}, - {0.36920,0.00,9.900,0,0.5440,6.5670,87.30,3.6023,4,304.0,18.40,395.69,9.28,23.80}, - {0.25356,0.00,9.900,0,0.5440,5.7050,77.70,3.9450,4,304.0,18.40,396.42,11.50,16.20}, - {0.31827,0.00,9.900,0,0.5440,5.9140,83.20,3.9986,4,304.0,18.40,390.70,18.33,17.80}, - {0.24522,0.00,9.900,0,0.5440,5.7820,71.70,4.0317,4,304.0,18.40,396.90,15.94,19.80}, - {0.40202,0.00,9.900,0,0.5440,6.3820,67.20,3.5325,4,304.0,18.40,395.21,10.36,23.10}, - {0.47547,0.00,9.900,0,0.5440,6.1130,58.80,4.0019,4,304.0,18.40,396.23,12.73,21.00}, - {0.16760,0.00,7.380,0,0.4930,6.4260,52.30,4.5404,5,287.0,19.60,396.90,7.20,23.80}, - {0.18159,0.00,7.380,0,0.4930,6.3760,54.30,4.5404,5,287.0,19.60,396.90,6.87,23.10}, - {0.35114,0.00,7.380,0,0.4930,6.0410,49.90,4.7211,5,287.0,19.60,396.90,7.70,20.40}, - {0.28392,0.00,7.380,0,0.4930,5.7080,74.30,4.7211,5,287.0,19.60,391.13,11.74,18.50}, - {0.34109,0.00,7.380,0,0.4930,6.4150,40.10,4.7211,5,287.0,19.60,396.90,6.12,25.00}, - {0.19186,0.00,7.380,0,0.4930,6.4310,14.70,5.4159,5,287.0,19.60,393.68,5.08,24.60}, - {0.30347,0.00,7.380,0,0.4930,6.3120,28.90,5.4159,5,287.0,19.60,396.90,6.15,23.00}, - {0.24103,0.00,7.380,0,0.4930,6.0830,43.70,5.4159,5,287.0,19.60,396.90,12.79,22.20}, - {0.06617,0.00,3.240,0,0.4600,5.8680,25.80,5.2146,4,430.0,16.90,382.44,9.97,19.30}, - {0.06724,0.00,3.240,0,0.4600,6.3330,17.20,5.2146,4,430.0,16.90,375.21,7.34,22.60}, - {0.04544,0.00,3.240,0,0.4600,6.1440,32.20,5.8736,4,430.0,16.90,368.57,9.09,19.80}, - {0.05023,35.00,6.060,0,0.4379,5.7060,28.40,6.6407,1,304.0,16.90,394.02,12.43,17.10}, - {0.03466,35.00,6.060,0,0.4379,6.0310,23.30,6.6407,1,304.0,16.90,362.25,7.83,19.40}, - {0.05083,0.00,5.190,0,0.5150,6.3160,38.10,6.4584,5,224.0,20.20,389.71,5.68,22.20}, - {0.03738,0.00,5.190,0,0.5150,6.3100,38.50,6.4584,5,224.0,20.20,389.40,6.75,20.70}, - {0.03961,0.00,5.190,0,0.5150,6.0370,34.50,5.9853,5,224.0,20.20,396.90,8.01,21.10}, - {0.03427,0.00,5.190,0,0.5150,5.8690,46.30,5.2311,5,224.0,20.20,396.90,9.80,19.50}, - {0.03041,0.00,5.190,0,0.5150,5.8950,59.60,5.6150,5,224.0,20.20,394.81,10.56,18.50}, - {0.03306,0.00,5.190,0,0.5150,6.0590,37.30,4.8122,5,224.0,20.20,396.14,8.51,20.60}, - {0.05497,0.00,5.190,0,0.5150,5.9850,45.40,4.8122,5,224.0,20.20,396.90,9.74,19.00}, - {0.06151,0.00,5.190,0,0.5150,5.9680,58.50,4.8122,5,224.0,20.20,396.90,9.29,18.70}, - {0.01301,35.00,1.520,0,0.4420,7.2410,49.30,7.0379,1,284.0,15.50,394.74,5.49,32.70}, - {0.02498,0.00,1.890,0,0.5180,6.5400,59.70,6.2669,1,422.0,15.90,389.96,8.65,16.50}, - {0.02543,55.00,3.780,0,0.4840,6.6960,56.40,5.7321,5,370.0,17.60,396.90,7.18,23.90}, - {0.03049,55.00,3.780,0,0.4840,6.8740,28.10,6.4654,5,370.0,17.60,387.97,4.61,31.20}, - {0.03113,0.00,4.390,0,0.4420,6.0140,48.50,8.0136,3,352.0,18.80,385.64,10.53,17.50}, - {0.06162,0.00,4.390,0,0.4420,5.8980,52.30,8.0136,3,352.0,18.80,364.61,12.67,17.20}, - {0.01870,85.00,4.150,0,0.4290,6.5160,27.70,8.5353,4,351.0,17.90,392.43,6.36,23.10}, - {0.01501,80.00,2.010,0,0.4350,6.6350,29.70,8.3440,4,280.0,17.00,390.94,5.99,24.50}, - {0.02899,40.00,1.250,0,0.4290,6.9390,34.50,8.7921,1,335.0,19.70,389.85,5.89,26.60}, - {0.06211,40.00,1.250,0,0.4290,6.4900,44.40,8.7921,1,335.0,19.70,396.90,5.98,22.90}, - {0.07950,60.00,1.690,0,0.4110,6.5790,35.90,10.7103,4,411.0,18.30,370.78,5.49,24.10}, - {0.07244,60.00,1.690,0,0.4110,5.8840,18.50,10.7103,4,411.0,18.30,392.33,7.79,18.60}, - {0.01709,90.00,2.020,0,0.4100,6.7280,36.10,12.1265,5,187.0,17.00,384.46,4.50,30.10}, - {0.04301,80.00,1.910,0,0.4130,5.6630,21.90,10.5857,4,334.0,22.00,382.80,8.05,18.20}, - {0.10659,80.00,1.910,0,0.4130,5.9360,19.50,10.5857,4,334.0,22.00,376.04,5.57,20.60}, - {8.98296,0.00,18.100,1,0.7700,6.2120,97.40,2.1222,24,666.0,20.20,377.73,17.60,17.80}, - {3.84970,0.00,18.100,1,0.7700,6.3950,91.00,2.5052,24,666.0,20.20,391.34,13.27,21.70}, - {5.20177,0.00,18.100,1,0.7700,6.1270,83.40,2.7227,24,666.0,20.20,395.43,11.48,22.70}, - {4.26131,0.00,18.100,0,0.7700,6.1120,81.30,2.5091,24,666.0,20.20,390.74,12.67,22.60}, - {4.54192,0.00,18.100,0,0.7700,6.3980,88.00,2.5182,24,666.0,20.20,374.56,7.79,25.00}, - {3.83684,0.00,18.100,0,0.7700,6.2510,91.10,2.2955,24,666.0,20.20,350.65,14.19,19.90}, - {3.67822,0.00,18.100,0,0.7700,5.3620,96.20,2.1036,24,666.0,20.20,380.79,10.19,20.80}, - {4.22239,0.00,18.100,1,0.7700,5.8030,89.00,1.9047,24,666.0,20.20,353.04,14.64,16.80}, - {3.47428,0.00,18.100,1,0.7180,8.7800,82.90,1.9047,24,666.0,20.20,354.55,5.29,21.90}, - {4.55587,0.00,18.100,0,0.7180,3.5610,87.90,1.6132,24,666.0,20.20,354.70,7.12,27.50}, - {3.69695,0.00,18.100,0,0.7180,4.9630,91.40,1.7523,24,666.0,20.20,316.03,14.00,21.90}, - {13.52220,0.00,18.100,0,0.6310,3.8630,100.00,1.5106,24,666.0,20.20,131.42,13.33,23.10}, - {4.89822,0.00,18.100,0,0.6310,4.9700,100.00,1.3325,24,666.0,20.20,375.52,3.26,50.00}, - {5.66998,0.00,18.100,1,0.6310,6.6830,96.80,1.3567,24,666.0,20.20,375.33,3.73,50.00}, - {6.53876,0.00,18.100,1,0.6310,7.0160,97.50,1.2024,24,666.0,20.20,392.05,2.96,50.00}, - {9.23230,0.00,18.100,0,0.6310,6.2160,100.00,1.1691,24,666.0,20.20,366.15,9.53,50.00}, - {8.26725,0.00,18.100,1,0.6680,5.8750,89.60,1.1296,24,666.0,20.20,347.88,8.88,50.00}, - {11.10810,0.00,18.100,0,0.6680,4.9060,100.00,1.1742,24,666.0,20.20,396.90,34.77,13.80}, - {18.49820,0.00,18.100,0,0.6680,4.1380,100.00,1.1370,24,666.0,20.20,396.90,37.97,13.80}, - {19.60910,0.00,18.100,0,0.6710,7.3130,97.90,1.3163,24,666.0,20.20,396.90,13.44,15.00}, - {15.28800,0.00,18.100,0,0.6710,6.6490,93.30,1.3449,24,666.0,20.20,363.02,23.24,13.90}, - {9.82349,0.00,18.100,0,0.6710,6.7940,98.80,1.3580,24,666.0,20.20,396.90,21.24,13.30}, - {23.64820,0.00,18.100,0,0.6710,6.3800,96.20,1.3861,24,666.0,20.20,396.90,23.69,13.10}, - {17.86670,0.00,18.100,0,0.6710,6.2230,100.00,1.3861,24,666.0,20.20,393.74,21.78,10.20}, - {88.97620,0.00,18.100,0,0.6710,6.9680,91.90,1.4165,24,666.0,20.20,396.90,17.21,10.40}, - {15.87440,0.00,18.100,0,0.6710,6.5450,99.10,1.5192,24,666.0,20.20,396.90,21.08,10.90}, - {9.18702,0.00,18.100,0,0.7000,5.5360,100.00,1.5804,24,666.0,20.20,396.90,23.60,11.30}, - {7.99248,0.00,18.100,0,0.7000,5.5200,100.00,1.5331,24,666.0,20.20,396.90,24.56,12.30}, - {20.08490,0.00,18.100,0,0.7000,4.3680,91.20,1.4395,24,666.0,20.20,285.83,30.63,8.80}, - {16.81180,0.00,18.100,0,0.7000,5.2770,98.10,1.4261,24,666.0,20.20,396.90,30.81,7.20}, - {24.39380,0.00,18.100,0,0.7000,4.6520,100.00,1.4672,24,666.0,20.20,396.90,28.28,10.50}, - {22.59710,0.00,18.100,0,0.7000,5.0000,89.50,1.5184,24,666.0,20.20,396.90,31.99,7.40}, - {14.33370,0.00,18.100,0,0.7000,4.8800,100.00,1.5895,24,666.0,20.20,372.92,30.62,10.20}, - {8.15174,0.00,18.100,0,0.7000,5.3900,98.90,1.7281,24,666.0,20.20,396.90,20.85,11.50}, - {6.96215,0.00,18.100,0,0.7000,5.7130,97.00,1.9265,24,666.0,20.20,394.43,17.11,15.10}, - {5.29305,0.00,18.100,0,0.7000,6.0510,82.50,2.1678,24,666.0,20.20,378.38,18.76,23.20}, - {11.57790,0.00,18.100,0,0.7000,5.0360,97.00,1.7700,24,666.0,20.20,396.90,25.68,9.70}, - {8.64476,0.00,18.100,0,0.6930,6.1930,92.60,1.7912,24,666.0,20.20,396.90,15.17,13.80}, - {13.35980,0.00,18.100,0,0.6930,5.8870,94.70,1.7821,24,666.0,20.20,396.90,16.35,12.70}, - {8.71675,0.00,18.100,0,0.6930,6.4710,98.80,1.7257,24,666.0,20.20,391.98,17.12,13.10}, - {5.87205,0.00,18.100,0,0.6930,6.4050,96.00,1.6768,24,666.0,20.20,396.90,19.37,12.50}, - {7.67202,0.00,18.100,0,0.6930,5.7470,98.90,1.6334,24,666.0,20.20,393.10,19.92,8.50}, - {38.35180,0.00,18.100,0,0.6930,5.4530,100.00,1.4896,24,666.0,20.20,396.90,30.59,5.00}, - {9.91655,0.00,18.100,0,0.6930,5.8520,77.80,1.5004,24,666.0,20.20,338.16,29.97,6.30}, - {25.04610,0.00,18.100,0,0.6930,5.9870,100.00,1.5888,24,666.0,20.20,396.90,26.77,5.60}, - {14.23620,0.00,18.100,0,0.6930,6.3430,100.00,1.5741,24,666.0,20.20,396.90,20.32,7.20}, - {9.59571,0.00,18.100,0,0.6930,6.4040,100.00,1.6390,24,666.0,20.20,376.11,20.31,12.10}, - {24.80170,0.00,18.100,0,0.6930,5.3490,96.00,1.7028,24,666.0,20.20,396.90,19.77,8.30}, - {41.52920,0.00,18.100,0,0.6930,5.5310,85.40,1.6074,24,666.0,20.20,329.46,27.38,8.50}, - {67.92080,0.00,18.100,0,0.6930,5.6830,100.00,1.4254,24,666.0,20.20,384.97,22.98,5.00}, - {20.71620,0.00,18.100,0,0.6590,4.1380,100.00,1.1781,24,666.0,20.20,370.22,23.34,11.90}, - {11.95110,0.00,18.100,0,0.6590,5.6080,100.00,1.2852,24,666.0,20.20,332.09,12.13,27.90}, - {7.40389,0.00,18.100,0,0.5970,5.6170,97.90,1.4547,24,666.0,20.20,314.64,26.40,17.20}, - {14.43830,0.00,18.100,0,0.5970,6.8520,100.00,1.4655,24,666.0,20.20,179.36,19.78,27.50}, - {51.13580,0.00,18.100,0,0.5970,5.7570,100.00,1.4130,24,666.0,20.20,2.60,10.11,15.00}, - {14.05070,0.00,18.100,0,0.5970,6.6570,100.00,1.5275,24,666.0,20.20,35.05,21.22,17.20}, - {18.81100,0.00,18.100,0,0.5970,4.6280,100.00,1.5539,24,666.0,20.20,28.79,34.37,17.90}, - {28.65580,0.00,18.100,0,0.5970,5.1550,100.00,1.5894,24,666.0,20.20,210.97,20.08,16.30}, - {45.74610,0.00,18.100,0,0.6930,4.5190,100.00,1.6582,24,666.0,20.20,88.27,36.98,7.00}, - {18.08460,0.00,18.100,0,0.6790,6.4340,100.00,1.8347,24,666.0,20.20,27.25,29.05,7.20}, - {10.83420,0.00,18.100,0,0.6790,6.7820,90.80,1.8195,24,666.0,20.20,21.57,25.79,7.50}, - {25.94060,0.00,18.100,0,0.6790,5.3040,89.10,1.6475,24,666.0,20.20,127.36,26.64,10.40}, - {73.53410,0.00,18.100,0,0.6790,5.9570,100.00,1.8026,24,666.0,20.20,16.45,20.62,8.80}, - {11.81230,0.00,18.100,0,0.7180,6.8240,76.50,1.7940,24,666.0,20.20,48.45,22.74,8.40}, - {11.08740,0.00,18.100,0,0.7180,6.4110,100.00,1.8589,24,666.0,20.20,318.75,15.02,16.70}, - {7.02259,0.00,18.100,0,0.7180,6.0060,95.30,1.8746,24,666.0,20.20,319.98,15.70,14.20}, - {12.04820,0.00,18.100,0,0.6140,5.6480,87.60,1.9512,24,666.0,20.20,291.55,14.10,20.80}, - {7.05042,0.00,18.100,0,0.6140,6.1030,85.10,2.0218,24,666.0,20.20,2.52,23.29,13.40}, - {8.79212,0.00,18.100,0,0.5840,5.5650,70.60,2.0635,24,666.0,20.20,3.65,17.16,11.70}, - {15.86030,0.00,18.100,0,0.6790,5.8960,95.40,1.9096,24,666.0,20.20,7.68,24.39,8.30}, - {12.24720,0.00,18.100,0,0.5840,5.8370,59.70,1.9976,24,666.0,20.20,24.65,15.69,10.20}, - {37.66190,0.00,18.100,0,0.6790,6.2020,78.70,1.8629,24,666.0,20.20,18.82,14.52,10.90}, - {7.36711,0.00,18.100,0,0.6790,6.1930,78.10,1.9356,24,666.0,20.20,96.73,21.52,11.00}, - {9.33889,0.00,18.100,0,0.6790,6.3800,95.60,1.9682,24,666.0,20.20,60.72,24.08,9.50}, - {8.49213,0.00,18.100,0,0.5840,6.3480,86.10,2.0527,24,666.0,20.20,83.45,17.64,14.50}, - {10.06230,0.00,18.100,0,0.5840,6.8330,94.30,2.0882,24,666.0,20.20,81.33,19.69,14.10}, - {6.44405,0.00,18.100,0,0.5840,6.4250,74.80,2.2004,24,666.0,20.20,97.95,12.03,16.10}, - {5.58107,0.00,18.100,0,0.7130,6.4360,87.90,2.3158,24,666.0,20.20,100.19,16.22,14.30}, - {13.91340,0.00,18.100,0,0.7130,6.2080,95.00,2.2222,24,666.0,20.20,100.63,15.17,11.70}, - {11.16040,0.00,18.100,0,0.7400,6.6290,94.60,2.1247,24,666.0,20.20,109.85,23.27,13.40}, - {14.42080,0.00,18.100,0,0.7400,6.4610,93.30,2.0026,24,666.0,20.20,27.49,18.05,9.60}, - {15.17720,0.00,18.100,0,0.7400,6.1520,100.00,1.9142,24,666.0,20.20,9.32,26.45,8.70}, - {13.67810,0.00,18.100,0,0.7400,5.9350,87.90,1.8206,24,666.0,20.20,68.95,34.02,8.40}, - {9.39063,0.00,18.100,0,0.7400,5.6270,93.90,1.8172,24,666.0,20.20,396.90,22.88,12.80}, - {22.05110,0.00,18.100,0,0.7400,5.8180,92.40,1.8662,24,666.0,20.20,391.45,22.11,10.50}, - {9.72418,0.00,18.100,0,0.7400,6.4060,97.20,2.0651,24,666.0,20.20,385.96,19.52,17.10}, - {5.66637,0.00,18.100,0,0.7400,6.2190,100.00,2.0048,24,666.0,20.20,395.69,16.59,18.40}, - {9.96654,0.00,18.100,0,0.7400,6.4850,100.00,1.9784,24,666.0,20.20,386.73,18.85,15.40}, - {12.80230,0.00,18.100,0,0.7400,5.8540,96.60,1.8956,24,666.0,20.20,240.52,23.79,10.80}, - {10.67180,0.00,18.100,0,0.7400,6.4590,94.80,1.9879,24,666.0,20.20,43.06,23.98,11.80}, - {6.28807,0.00,18.100,0,0.7400,6.3410,96.40,2.0720,24,666.0,20.20,318.01,17.79,14.90}, - {9.92485,0.00,18.100,0,0.7400,6.2510,96.60,2.1980,24,666.0,20.20,388.52,16.44,12.60}, - {9.32909,0.00,18.100,0,0.7130,6.1850,98.70,2.2616,24,666.0,20.20,396.90,18.13,14.10}, - {7.52601,0.00,18.100,0,0.7130,6.4170,98.30,2.1850,24,666.0,20.20,304.21,19.31,13.00}, - {6.71772,0.00,18.100,0,0.7130,6.7490,92.60,2.3236,24,666.0,20.20,0.32,17.44,13.40}, - {5.44114,0.00,18.100,0,0.7130,6.6550,98.20,2.3552,24,666.0,20.20,355.29,17.73,15.20}, - {5.09017,0.00,18.100,0,0.7130,6.2970,91.80,2.3682,24,666.0,20.20,385.09,17.27,16.10}, - {8.24809,0.00,18.100,0,0.7130,7.3930,99.30,2.4527,24,666.0,20.20,375.87,16.74,17.80}, - {9.51363,0.00,18.100,0,0.7130,6.7280,94.10,2.4961,24,666.0,20.20,6.68,18.71,14.90}, - {4.75237,0.00,18.100,0,0.7130,6.5250,86.50,2.4358,24,666.0,20.20,50.92,18.13,14.10}, - {4.66883,0.00,18.100,0,0.7130,5.9760,87.90,2.5806,24,666.0,20.20,10.48,19.01,12.70}, - {8.20058,0.00,18.100,0,0.7130,5.9360,80.30,2.7792,24,666.0,20.20,3.50,16.94,13.50}, - {7.75223,0.00,18.100,0,0.7130,6.3010,83.70,2.7831,24,666.0,20.20,272.21,16.23,14.90}, - {6.80117,0.00,18.100,0,0.7130,6.0810,84.40,2.7175,24,666.0,20.20,396.90,14.70,20.00}, - {4.81213,0.00,18.100,0,0.7130,6.7010,90.00,2.5975,24,666.0,20.20,255.23,16.42,16.40}, - {3.69311,0.00,18.100,0,0.7130,6.3760,88.40,2.5671,24,666.0,20.20,391.43,14.65,17.70}, - {6.65492,0.00,18.100,0,0.7130,6.3170,83.00,2.7344,24,666.0,20.20,396.90,13.99,19.50}, - {5.82115,0.00,18.100,0,0.7130,6.5130,89.90,2.8016,24,666.0,20.20,393.82,10.29,20.20}, - {7.83932,0.00,18.100,0,0.6550,6.2090,65.40,2.9634,24,666.0,20.20,396.90,13.22,21.40}, - {3.16360,0.00,18.100,0,0.6550,5.7590,48.20,3.0665,24,666.0,20.20,334.40,14.13,19.90}, - {3.77498,0.00,18.100,0,0.6550,5.9520,84.70,2.8715,24,666.0,20.20,22.01,17.15,19.00}, - {4.42228,0.00,18.100,0,0.5840,6.0030,94.50,2.5403,24,666.0,20.20,331.29,21.32,19.10}, - {15.57570,0.00,18.100,0,0.5800,5.9260,71.00,2.9084,24,666.0,20.20,368.74,18.13,19.10}, - {13.07510,0.00,18.100,0,0.5800,5.7130,56.70,2.8237,24,666.0,20.20,396.90,14.76,20.10}, - {4.34879,0.00,18.100,0,0.5800,6.1670,84.00,3.0334,24,666.0,20.20,396.90,16.29,19.90}, - {4.03841,0.00,18.100,0,0.5320,6.2290,90.70,3.0993,24,666.0,20.20,395.33,12.87,19.60}, - {3.56868,0.00,18.100,0,0.5800,6.4370,75.00,2.8965,24,666.0,20.20,393.37,14.36,23.20}, - {4.64689,0.00,18.100,0,0.6140,6.9800,67.60,2.5329,24,666.0,20.20,374.68,11.66,29.80}, - {8.05579,0.00,18.100,0,0.5840,5.4270,95.40,2.4298,24,666.0,20.20,352.58,18.14,13.80}, - {6.39312,0.00,18.100,0,0.5840,6.1620,97.40,2.2060,24,666.0,20.20,302.76,24.10,13.30}, - {4.87141,0.00,18.100,0,0.6140,6.4840,93.60,2.3053,24,666.0,20.20,396.21,18.68,16.70}, - {15.02340,0.00,18.100,0,0.6140,5.3040,97.30,2.1007,24,666.0,20.20,349.48,24.91,12.00}, - {10.23300,0.00,18.100,0,0.6140,6.1850,96.70,2.1705,24,666.0,20.20,379.70,18.03,14.60}, - {14.33370,0.00,18.100,0,0.6140,6.2290,88.00,1.9512,24,666.0,20.20,383.32,13.11,21.40}, - {5.82401,0.00,18.100,0,0.5320,6.2420,64.70,3.4242,24,666.0,20.20,396.90,10.74,23.00}, - {5.70818,0.00,18.100,0,0.5320,6.7500,74.90,3.3317,24,666.0,20.20,393.07,7.74,23.70}, - {5.73116,0.00,18.100,0,0.5320,7.0610,77.00,3.4106,24,666.0,20.20,395.28,7.01,25.00}, - {2.81838,0.00,18.100,0,0.5320,5.7620,40.30,4.0983,24,666.0,20.20,392.92,10.42,21.80}, - {2.37857,0.00,18.100,0,0.5830,5.8710,41.90,3.7240,24,666.0,20.20,370.73,13.34,20.60}, - {3.67367,0.00,18.100,0,0.5830,6.3120,51.90,3.9917,24,666.0,20.20,388.62,10.58,21.20}, - {5.69175,0.00,18.100,0,0.5830,6.1140,79.80,3.5459,24,666.0,20.20,392.68,14.98,19.10}, - {4.83567,0.00,18.100,0,0.5830,5.9050,53.20,3.1523,24,666.0,20.20,388.22,11.45,20.60}, - {0.15086,0.00,27.740,0,0.6090,5.4540,92.70,1.8209,4,711.0,20.10,395.09,18.06,15.20}, - {0.18337,0.00,27.740,0,0.6090,5.4140,98.30,1.7554,4,711.0,20.10,344.05,23.97,7.00}, - {0.20746,0.00,27.740,0,0.6090,5.0930,98.00,1.8226,4,711.0,20.10,318.43,29.68,8.10}, - {0.10574,0.00,27.740,0,0.6090,5.9830,98.80,1.8681,4,711.0,20.10,390.11,18.07,13.60}, - {0.11132,0.00,27.740,0,0.6090,5.9830,83.50,2.1099,4,711.0,20.10,396.90,13.35,20.10}, - {0.17331,0.00,9.690,0,0.5850,5.7070,54.00,2.3817,6,391.0,19.20,396.90,12.01,21.80}, - {0.27957,0.00,9.690,0,0.5850,5.9260,42.60,2.3817,6,391.0,19.20,396.90,13.59,24.50}, - {0.17899,0.00,9.690,0,0.5850,5.6700,28.80,2.7986,6,391.0,19.20,393.29,17.60,23.10}, - {0.28960,0.00,9.690,0,0.5850,5.3900,72.90,2.7986,6,391.0,19.20,396.90,21.14,19.70}, - {0.26838,0.00,9.690,0,0.5850,5.7940,70.60,2.8927,6,391.0,19.20,396.90,14.10,18.30}, - {0.23912,0.00,9.690,0,0.5850,6.0190,65.30,2.4091,6,391.0,19.20,396.90,12.92,21.20}, - {0.17783,0.00,9.690,0,0.5850,5.5690,73.50,2.3999,6,391.0,19.20,395.77,15.10,17.50}, - {0.22438,0.00,9.690,0,0.5850,6.0270,79.70,2.4982,6,391.0,19.20,396.90,14.33,16.80}, - {0.06263,0.00,11.930,0,0.5730,6.5930,69.10,2.4786,1,273.0,21.00,391.99,9.67,22.40}, - {0.04527,0.00,11.930,0,0.5730,6.1200,76.70,2.2875,1,273.0,21.00,396.90,9.08,20.60}, - {0.06076,0.00,11.930,0,0.5730,6.9760,91.00,2.1675,1,273.0,21.00,396.90,5.64,23.90}, - {0.10959,0.00,11.930,0,0.5730,6.7940,89.30,2.3889,1,273.0,21.00,393.45,6.48,22.00}, - {0.04741,0.00,11.930,0,0.5730,6.0300,80.80,2.5050,1,273.0,21.00,396.90,7.88,11.90} + {0.02731,0.00,7.070,0,0.4690,6.4210,78.90,4.9671,2,242.0,17.80,396.90,9.14,21.60}, + {0.02729,0.00,7.070,0,0.4690,7.1850,61.10,4.9671,2,242.0,17.80,392.83,4.03,34.70}, + {0.03237,0.00,2.180,0,0.4580,6.9980,45.80,6.0622,3,222.0,18.70,394.63,2.94,33.40}, + {0.06905,0.00,2.180,0,0.4580,7.1470,54.20,6.0622,3,222.0,18.70,396.90,5.33,36.20}, + {0.02985,0.00,2.180,0,0.4580,6.4300,58.70,6.0622,3,222.0,18.70,394.12,5.21,28.70}, + {0.08829,12.50,7.870,0,0.5240,6.0120,66.60,5.5605,5,311.0,15.20,395.60,12.43,22.90}, + {0.14455,12.50,7.870,0,0.5240,6.1720,96.10,5.9505,5,311.0,15.20,396.90,19.15,27.10}, + {0.21124,12.50,7.870,0,0.5240,5.6310,100.00,6.0821,5,311.0,15.20,386.63,29.93,16.50}, + {0.17004,12.50,7.870,0,0.5240,6.0040,85.90,6.5921,5,311.0,15.20,386.71,17.10,18.90}, + {0.22489,12.50,7.870,0,0.5240,6.3770,94.30,6.3467,5,311.0,15.20,392.52,20.45,15.00}, + {0.11747,12.50,7.870,0,0.5240,6.0090,82.90,6.2267,5,311.0,15.20,396.90,13.27,18.90}, + {0.09378,12.50,7.870,0,0.5240,5.8890,39.00,5.4509,5,311.0,15.20,390.50,15.71,21.70}, + {0.62976,0.00,8.140,0,0.5380,5.9490,61.80,4.7075,4,307.0,21.00,396.90,8.26,20.40}, + {0.63796,0.00,8.140,0,0.5380,6.0960,84.50,4.4619,4,307.0,21.00,380.02,10.26,18.20}, + {0.62739,0.00,8.140,0,0.5380,5.8340,56.50,4.4986,4,307.0,21.00,395.62,8.47,19.90}, + {1.05393,0.00,8.140,0,0.5380,5.9350,29.30,4.4986,4,307.0,21.00,386.85,6.58,23.10}, + {0.78420,0.00,8.140,0,0.5380,5.9900,81.70,4.2579,4,307.0,21.00,386.75,14.67,17.50}, + {0.80271,0.00,8.140,0,0.5380,5.4560,36.60,3.7965,4,307.0,21.00,288.99,11.69,20.20}, + {0.72580,0.00,8.140,0,0.5380,5.7270,69.50,3.7965,4,307.0,21.00,390.95,11.28,18.20}, + {1.25179,0.00,8.140,0,0.5380,5.5700,98.10,3.7979,4,307.0,21.00,376.57,21.02,13.60}, + {0.85204,0.00,8.140,0,0.5380,5.9650,89.20,4.0123,4,307.0,21.00,392.53,13.83,19.60}, + {1.23247,0.00,8.140,0,0.5380,6.1420,91.70,3.9769,4,307.0,21.00,396.90,18.72,15.20}, + {0.98843,0.00,8.140,0,0.5380,5.8130,100.00,4.0952,4,307.0,21.00,394.54,19.88,14.50}, + {0.75026,0.00,8.140,0,0.5380,5.9240,94.10,4.3996,4,307.0,21.00,394.33,16.30,15.60}, + {0.84054,0.00,8.140,0,0.5380,5.5990,85.70,4.4546,4,307.0,21.00,303.42,16.51,13.90}, + {0.67191,0.00,8.140,0,0.5380,5.8130,90.30,4.6820,4,307.0,21.00,376.88,14.81,16.60}, + {0.95577,0.00,8.140,0,0.5380,6.0470,88.80,4.4534,4,307.0,21.00,306.38,17.28,14.80}, + {0.77299,0.00,8.140,0,0.5380,6.4950,94.40,4.4547,4,307.0,21.00,387.94,12.80,18.40}, + {1.00245,0.00,8.140,0,0.5380,6.6740,87.30,4.2390,4,307.0,21.00,380.23,11.98,21.00}, + {1.13081,0.00,8.140,0,0.5380,5.7130,94.10,4.2330,4,307.0,21.00,360.17,22.60,12.70}, + {1.35472,0.00,8.140,0,0.5380,6.0720,100.00,4.1750,4,307.0,21.00,376.73,13.04,14.50}, + {1.38799,0.00,8.140,0,0.5380,5.9500,82.00,3.9900,4,307.0,21.00,232.60,27.71,13.20}, + {1.15172,0.00,8.140,0,0.5380,5.7010,95.00,3.7872,4,307.0,21.00,358.77,18.35,13.10}, + {1.61282,0.00,8.140,0,0.5380,6.0960,96.90,3.7598,4,307.0,21.00,248.31,20.34,13.50}, + {0.06417,0.00,5.960,0,0.4990,5.9330,68.20,3.3603,5,279.0,19.20,396.90,9.68,18.90}, + {0.09744,0.00,5.960,0,0.4990,5.8410,61.40,3.3779,5,279.0,19.20,377.56,11.41,20.00}, + {0.08014,0.00,5.960,0,0.4990,5.8500,41.50,3.9342,5,279.0,19.20,396.90,8.77,21.00}, + {0.17505,0.00,5.960,0,0.4990,5.9660,30.20,3.8473,5,279.0,19.20,393.43,10.13,24.70}, + {0.02763,75.00,2.950,0,0.4280,6.5950,21.80,5.4011,3,252.0,18.30,395.63,4.32,30.80}, + {0.03359,75.00,2.950,0,0.4280,7.0240,15.80,5.4011,3,252.0,18.30,395.62,1.98,34.90}, + {0.12744,0.00,6.910,0,0.4480,6.7700,2.90,5.7209,3,233.0,17.90,385.41,4.84,26.60}, + {0.14150,0.00,6.910,0,0.4480,6.1690,6.60,5.7209,3,233.0,17.90,383.37,5.81,25.30}, + {0.15936,0.00,6.910,0,0.4480,6.2110,6.50,5.7209,3,233.0,17.90,394.46,7.44,24.70}, + {0.12269,0.00,6.910,0,0.4480,6.0690,40.00,5.7209,3,233.0,17.90,389.39,9.55,21.20}, + {0.17142,0.00,6.910,0,0.4480,5.6820,33.80,5.1004,3,233.0,17.90,396.90,10.21,19.30}, + {0.18836,0.00,6.910,0,0.4480,5.7860,33.30,5.1004,3,233.0,17.90,396.90,14.15,20.00}, + {0.22927,0.00,6.910,0,0.4480,6.0300,85.50,5.6894,3,233.0,17.90,392.74,18.80,16.60}, + {0.25387,0.00,6.910,0,0.4480,5.3990,95.30,5.8700,3,233.0,17.90,396.90,30.81,14.40}, + {0.21977,0.00,6.910,0,0.4480,5.6020,62.00,6.0877,3,233.0,17.90,396.90,16.20,19.40}, + {0.08873,21.00,5.640,0,0.4390,5.9630,45.70,6.8147,4,243.0,16.80,395.56,13.45,19.70}, + {0.04337,21.00,5.640,0,0.4390,6.1150,63.00,6.8147,4,243.0,16.80,393.97,9.43,20.50}, + {0.05360,21.00,5.640,0,0.4390,6.5110,21.10,6.8147,4,243.0,16.80,396.90,5.28,25.00}, + {0.04981,21.00,5.640,0,0.4390,5.9980,21.40,6.8147,4,243.0,16.80,396.90,8.43,23.40}, + {0.01360,75.00,4.000,0,0.4100,5.8880,47.60,7.3197,3,469.0,21.10,396.90,14.80,18.90}, + {0.01311,90.00,1.220,0,0.4030,7.2490,21.90,8.6966,5,226.0,17.90,395.93,4.81,35.40}, + {0.02055,85.00,0.740,0,0.4100,6.3830,35.70,9.1876,2,313.0,17.30,396.90,5.77,24.70}, + {0.01432,100.00,1.320,0,0.4110,6.8160,40.50,8.3248,5,256.0,15.10,392.90,3.95,31.60}, + {0.15445,25.00,5.130,0,0.4530,6.1450,29.20,7.8148,8,284.0,19.70,390.68,6.86,23.30}, + {0.10328,25.00,5.130,0,0.4530,5.9270,47.20,6.9320,8,284.0,19.70,396.90,9.22,19.60}, + {0.14932,25.00,5.130,0,0.4530,5.7410,66.20,7.2254,8,284.0,19.70,395.11,13.15,18.70}, + {0.17171,25.00,5.130,0,0.4530,5.9660,93.40,6.8185,8,284.0,19.70,378.08,14.44,16.00}, + {0.11027,25.00,5.130,0,0.4530,6.4560,67.80,7.2255,8,284.0,19.70,396.90,6.73,22.20}, + {0.12650,25.00,5.130,0,0.4530,6.7620,43.40,7.9809,8,284.0,19.70,395.58,9.50,25.00}, + {0.01951,17.50,1.380,0,0.4161,7.1040,59.50,9.2229,3,216.0,18.60,393.24,8.05,33.00}, + {0.03584,80.00,3.370,0,0.3980,6.2900,17.80,6.6115,4,337.0,16.10,396.90,4.67,23.50}, + {0.04379,80.00,3.370,0,0.3980,5.7870,31.10,6.6115,4,337.0,16.10,396.90,10.24,19.40}, + {0.05789,12.50,6.070,0,0.4090,5.8780,21.40,6.4980,4,345.0,18.90,396.21,8.10,22.00}, + {0.13554,12.50,6.070,0,0.4090,5.5940,36.80,6.4980,4,345.0,18.90,396.90,13.09,17.40}, + {0.12816,12.50,6.070,0,0.4090,5.8850,33.00,6.4980,4,345.0,18.90,396.90,8.79,20.90}, + {0.08826,0.00,10.810,0,0.4130,6.4170,6.60,5.2873,4,305.0,19.20,383.73,6.72,24.20}, + {0.15876,0.00,10.810,0,0.4130,5.9610,17.50,5.2873,4,305.0,19.20,376.94,9.88,21.70}, + {0.09164,0.00,10.810,0,0.4130,6.0650,7.80,5.2873,4,305.0,19.20,390.91,5.52,22.80}, + {0.19539,0.00,10.810,0,0.4130,6.2450,6.20,5.2873,4,305.0,19.20,377.17,7.54,23.40}, + {0.07896,0.00,12.830,0,0.4370,6.2730,6.00,4.2515,5,398.0,18.70,394.92,6.78,24.10}, + {0.09512,0.00,12.830,0,0.4370,6.2860,45.00,4.5026,5,398.0,18.70,383.23,8.94,21.40}, + {0.10153,0.00,12.830,0,0.4370,6.2790,74.50,4.0522,5,398.0,18.70,373.66,11.97,20.00}, + {0.08707,0.00,12.830,0,0.4370,6.1400,45.80,4.0905,5,398.0,18.70,386.96,10.27,20.80}, + {0.05646,0.00,12.830,0,0.4370,6.2320,53.70,5.0141,5,398.0,18.70,386.40,12.34,21.20}, + {0.08387,0.00,12.830,0,0.4370,5.8740,36.60,4.5026,5,398.0,18.70,396.06,9.10,20.30}, + {0.04113,25.00,4.860,0,0.4260,6.7270,33.50,5.4007,4,281.0,19.00,396.90,5.29,28.00}, + {0.04462,25.00,4.860,0,0.4260,6.6190,70.40,5.4007,4,281.0,19.00,395.63,7.22,23.90}, + {0.03659,25.00,4.860,0,0.4260,6.3020,32.20,5.4007,4,281.0,19.00,396.90,6.72,24.80}, + {0.03551,25.00,4.860,0,0.4260,6.1670,46.70,5.4007,4,281.0,19.00,390.64,7.51,22.90}, + {0.05059,0.00,4.490,0,0.4490,6.3890,48.00,4.7794,3,247.0,18.50,396.90,9.62,23.90}, + {0.05735,0.00,4.490,0,0.4490,6.6300,56.10,4.4377,3,247.0,18.50,392.30,6.53,26.60}, + {0.05188,0.00,4.490,0,0.4490,6.0150,45.10,4.4272,3,247.0,18.50,395.99,12.86,22.50}, + {0.07151,0.00,4.490,0,0.4490,6.1210,56.80,3.7476,3,247.0,18.50,395.15,8.44,22.20}, + {0.05660,0.00,3.410,0,0.4890,7.0070,86.30,3.4217,2,270.0,17.80,396.90,5.50,23.60}, + {0.05302,0.00,3.410,0,0.4890,7.0790,63.10,3.4145,2,270.0,17.80,396.06,5.70,28.70}, + {0.04684,0.00,3.410,0,0.4890,6.4170,66.10,3.0923,2,270.0,17.80,392.18,8.81,22.60}, + {0.03932,0.00,3.410,0,0.4890,6.4050,73.90,3.0921,2,270.0,17.80,393.55,8.20,22.00}, + {0.04203,28.00,15.040,0,0.4640,6.4420,53.60,3.6659,4,270.0,18.20,395.01,8.16,22.90}, + {0.02875,28.00,15.040,0,0.4640,6.2110,28.90,3.6659,4,270.0,18.20,396.33,6.21,25.00}, + {0.04294,28.00,15.040,0,0.4640,6.2490,77.30,3.6150,4,270.0,18.20,396.90,10.59,20.60}, + {0.12204,0.00,2.890,0,0.4450,6.6250,57.80,3.4952,2,276.0,18.00,357.98,6.65,28.40}, + {0.11504,0.00,2.890,0,0.4450,6.1630,69.60,3.4952,2,276.0,18.00,391.83,11.34,21.40}, + {0.12083,0.00,2.890,0,0.4450,8.0690,76.00,3.4952,2,276.0,18.00,396.90,4.21,38.70}, + {0.08187,0.00,2.890,0,0.4450,7.8200,36.90,3.4952,2,276.0,18.00,393.53,3.57,43.80}, + {0.06860,0.00,2.890,0,0.4450,7.4160,62.50,3.4952,2,276.0,18.00,396.90,6.19,33.20}, + {0.14866,0.00,8.560,0,0.5200,6.7270,79.90,2.7778,5,384.0,20.90,394.76,9.42,27.50}, + {0.11432,0.00,8.560,0,0.5200,6.7810,71.30,2.8561,5,384.0,20.90,395.58,7.67,26.50}, + {0.22876,0.00,8.560,0,0.5200,6.4050,85.40,2.7147,5,384.0,20.90,70.80,10.63,18.60}, + {0.21161,0.00,8.560,0,0.5200,6.1370,87.40,2.7147,5,384.0,20.90,394.47,13.44,19.30}, + {0.13960,0.00,8.560,0,0.5200,6.1670,90.00,2.4210,5,384.0,20.90,392.69,12.33,20.10}, + {0.13262,0.00,8.560,0,0.5200,5.8510,96.70,2.1069,5,384.0,20.90,394.05,16.47,19.50}, + {0.17120,0.00,8.560,0,0.5200,5.8360,91.90,2.2110,5,384.0,20.90,395.67,18.66,19.50}, + {0.13117,0.00,8.560,0,0.5200,6.1270,85.20,2.1224,5,384.0,20.90,387.69,14.09,20.40}, + {0.12802,0.00,8.560,0,0.5200,6.4740,97.10,2.4329,5,384.0,20.90,395.24,12.27,19.80}, + {0.26363,0.00,8.560,0,0.5200,6.2290,91.20,2.5451,5,384.0,20.90,391.23,15.55,19.40}, + {0.10793,0.00,8.560,0,0.5200,6.1950,54.40,2.7778,5,384.0,20.90,393.49,13.00,21.70}, + {0.10084,0.00,10.010,0,0.5470,6.7150,81.60,2.6775,6,432.0,17.80,395.59,10.16,22.80}, + {0.12329,0.00,10.010,0,0.5470,5.9130,92.90,2.3534,6,432.0,17.80,394.95,16.21,18.80}, + {0.22212,0.00,10.010,0,0.5470,6.0920,95.40,2.5480,6,432.0,17.80,396.90,17.09,18.70}, + {0.14231,0.00,10.010,0,0.5470,6.2540,84.20,2.2565,6,432.0,17.80,388.74,10.45,18.50}, + {0.17134,0.00,10.010,0,0.5470,5.9280,88.20,2.4631,6,432.0,17.80,344.91,15.76,18.30}, + {0.13158,0.00,10.010,0,0.5470,6.1760,72.50,2.7301,6,432.0,17.80,393.30,12.04,21.20}, + {0.15098,0.00,10.010,0,0.5470,6.0210,82.60,2.7474,6,432.0,17.80,394.51,10.30,19.20}, + {0.13058,0.00,10.010,0,0.5470,5.8720,73.10,2.4775,6,432.0,17.80,338.63,15.37,20.40}, + {0.14476,0.00,10.010,0,0.5470,5.7310,65.20,2.7592,6,432.0,17.80,391.50,13.61,19.30}, + {0.06899,0.00,25.650,0,0.5810,5.8700,69.70,2.2577,2,188.0,19.10,389.15,14.37,22.00}, + {0.07165,0.00,25.650,0,0.5810,6.0040,84.10,2.1974,2,188.0,19.10,377.67,14.27,20.30}, + {0.09299,0.00,25.650,0,0.5810,5.9610,92.90,2.0869,2,188.0,19.10,378.09,17.93,20.50}, + {0.15038,0.00,25.650,0,0.5810,5.8560,97.00,1.9444,2,188.0,19.10,370.31,25.41,17.30}, + {0.09849,0.00,25.650,0,0.5810,5.8790,95.80,2.0063,2,188.0,19.10,379.38,17.58,18.80}, + {0.16902,0.00,25.650,0,0.5810,5.9860,88.40,1.9929,2,188.0,19.10,385.02,14.81,21.40}, + {0.38735,0.00,25.650,0,0.5810,5.6130,95.60,1.7572,2,188.0,19.10,359.29,27.26,15.70}, + {0.25915,0.00,21.890,0,0.6240,5.6930,96.00,1.7883,4,437.0,21.20,392.11,17.19,16.20}, + {0.32543,0.00,21.890,0,0.6240,6.4310,98.80,1.8125,4,437.0,21.20,396.90,15.39,18.00}, + {0.88125,0.00,21.890,0,0.6240,5.6370,94.70,1.9799,4,437.0,21.20,396.90,18.34,14.30}, + {0.34006,0.00,21.890,0,0.6240,6.4580,98.90,2.1185,4,437.0,21.20,395.04,12.60,19.20}, + {1.19294,0.00,21.890,0,0.6240,6.3260,97.70,2.2710,4,437.0,21.20,396.90,12.26,19.60}, + {0.59005,0.00,21.890,0,0.6240,6.3720,97.90,2.3274,4,437.0,21.20,385.76,11.12,23.00}, + {0.32982,0.00,21.890,0,0.6240,5.8220,95.40,2.4699,4,437.0,21.20,388.69,15.03,18.40}, + {0.97617,0.00,21.890,0,0.6240,5.7570,98.40,2.3460,4,437.0,21.20,262.76,17.31,15.60}, + {0.55778,0.00,21.890,0,0.6240,6.3350,98.20,2.1107,4,437.0,21.20,394.67,16.96,18.10}, + {0.32264,0.00,21.890,0,0.6240,5.9420,93.50,1.9669,4,437.0,21.20,378.25,16.90,17.40}, + {0.35233,0.00,21.890,0,0.6240,6.4540,98.40,1.8498,4,437.0,21.20,394.08,14.59,17.10}, + {0.24980,0.00,21.890,0,0.6240,5.8570,98.20,1.6686,4,437.0,21.20,392.04,21.32,13.30}, + {0.54452,0.00,21.890,0,0.6240,6.1510,97.90,1.6687,4,437.0,21.20,396.90,18.46,17.80}, + {0.29090,0.00,21.890,0,0.6240,6.1740,93.60,1.6119,4,437.0,21.20,388.08,24.16,14.00}, + {1.62864,0.00,21.890,0,0.6240,5.0190,100.00,1.4394,4,437.0,21.20,396.90,34.41,14.40}, + {3.32105,0.00,19.580,1,0.8710,5.4030,100.00,1.3216,5,403.0,14.70,396.90,26.82,13.40}, + {4.09740,0.00,19.580,0,0.8710,5.4680,100.00,1.4118,5,403.0,14.70,396.90,26.42,15.60}, + {2.77974,0.00,19.580,0,0.8710,4.9030,97.80,1.3459,5,403.0,14.70,396.90,29.29,11.80}, + {2.37934,0.00,19.580,0,0.8710,6.1300,100.00,1.4191,5,403.0,14.70,172.91,27.80,13.80}, + {2.15505,0.00,19.580,0,0.8710,5.6280,100.00,1.5166,5,403.0,14.70,169.27,16.65,15.60}, + {2.36862,0.00,19.580,0,0.8710,4.9260,95.70,1.4608,5,403.0,14.70,391.71,29.53,14.60}, + {2.33099,0.00,19.580,0,0.8710,5.1860,93.80,1.5296,5,403.0,14.70,356.99,28.32,17.80}, + {2.73397,0.00,19.580,0,0.8710,5.5970,94.90,1.5257,5,403.0,14.70,351.85,21.45,15.40}, + {1.65660,0.00,19.580,0,0.8710,6.1220,97.30,1.6180,5,403.0,14.70,372.80,14.10,21.50}, + {1.49632,0.00,19.580,0,0.8710,5.4040,100.00,1.5916,5,403.0,14.70,341.60,13.28,19.60}, + {1.12658,0.00,19.580,1,0.8710,5.0120,88.00,1.6102,5,403.0,14.70,343.28,12.12,15.30}, + {2.14918,0.00,19.580,0,0.8710,5.7090,98.50,1.6232,5,403.0,14.70,261.95,15.79,19.40}, + {1.41385,0.00,19.580,1,0.8710,6.1290,96.00,1.7494,5,403.0,14.70,321.02,15.12,17.00}, + {3.53501,0.00,19.580,1,0.8710,6.1520,82.60,1.7455,5,403.0,14.70,88.01,15.02,15.60}, + {2.44668,0.00,19.580,0,0.8710,5.2720,94.00,1.7364,5,403.0,14.70,88.63,16.14,13.10}, + {1.22358,0.00,19.580,0,0.6050,6.9430,97.40,1.8773,5,403.0,14.70,363.43,4.59,41.30}, + {1.34284,0.00,19.580,0,0.6050,6.0660,100.00,1.7573,5,403.0,14.70,353.89,6.43,24.30}, + {1.42502,0.00,19.580,0,0.8710,6.5100,100.00,1.7659,5,403.0,14.70,364.31,7.39,23.30}, + {1.27346,0.00,19.580,1,0.6050,6.2500,92.60,1.7984,5,403.0,14.70,338.92,5.50,27.00}, + {1.46336,0.00,19.580,0,0.6050,7.4890,90.80,1.9709,5,403.0,14.70,374.43,1.73,50.00}, + {1.83377,0.00,19.580,1,0.6050,7.8020,98.20,2.0407,5,403.0,14.70,389.61,1.92,50.00}, + {1.51902,0.00,19.580,1,0.6050,8.3750,93.90,2.1620,5,403.0,14.70,388.45,3.32,50.00}, + {2.24236,0.00,19.580,0,0.6050,5.8540,91.80,2.4220,5,403.0,14.70,395.11,11.64,22.70}, + {2.92400,0.00,19.580,0,0.6050,6.1010,93.00,2.2834,5,403.0,14.70,240.16,9.81,25.00}, + {2.01019,0.00,19.580,0,0.6050,7.9290,96.20,2.0459,5,403.0,14.70,369.30,3.70,50.00}, + {1.80028,0.00,19.580,0,0.6050,5.8770,79.20,2.4259,5,403.0,14.70,227.61,12.14,23.80}, + {2.30040,0.00,19.580,0,0.6050,6.3190,96.10,2.1000,5,403.0,14.70,297.09,11.10,23.80}, + {2.44953,0.00,19.580,0,0.6050,6.4020,95.20,2.2625,5,403.0,14.70,330.04,11.32,22.30}, + {1.20742,0.00,19.580,0,0.6050,5.8750,94.60,2.4259,5,403.0,14.70,292.29,14.43,17.40}, + {2.31390,0.00,19.580,0,0.6050,5.8800,97.30,2.3887,5,403.0,14.70,348.13,12.03,19.10}, + {0.13914,0.00,4.050,0,0.5100,5.5720,88.50,2.5961,5,296.0,16.60,396.90,14.69,23.10}, + {0.09178,0.00,4.050,0,0.5100,6.4160,84.10,2.6463,5,296.0,16.60,395.50,9.04,23.60}, + {0.08447,0.00,4.050,0,0.5100,5.8590,68.70,2.7019,5,296.0,16.60,393.23,9.64,22.60}, + {0.06664,0.00,4.050,0,0.5100,6.5460,33.10,3.1323,5,296.0,16.60,390.96,5.33,29.40}, + {0.07022,0.00,4.050,0,0.5100,6.0200,47.20,3.5549,5,296.0,16.60,393.23,10.11,23.20}, + {0.05425,0.00,4.050,0,0.5100,6.3150,73.40,3.3175,5,296.0,16.60,395.60,6.29,24.60}, + {0.06642,0.00,4.050,0,0.5100,6.8600,74.40,2.9153,5,296.0,16.60,391.27,6.92,29.90}, + {0.05780,0.00,2.460,0,0.4880,6.9800,58.40,2.8290,3,193.0,17.80,396.90,5.04,37.20}, + {0.06588,0.00,2.460,0,0.4880,7.7650,83.30,2.7410,3,193.0,17.80,395.56,7.56,39.80}, + {0.06888,0.00,2.460,0,0.4880,6.1440,62.20,2.5979,3,193.0,17.80,396.90,9.45,36.20}, + {0.09103,0.00,2.460,0,0.4880,7.1550,92.20,2.7006,3,193.0,17.80,394.12,4.82,37.90}, + {0.10008,0.00,2.460,0,0.4880,6.5630,95.60,2.8470,3,193.0,17.80,396.90,5.68,32.50}, + {0.08308,0.00,2.460,0,0.4880,5.6040,89.80,2.9879,3,193.0,17.80,391.00,13.98,26.40}, + {0.06047,0.00,2.460,0,0.4880,6.1530,68.80,3.2797,3,193.0,17.80,387.11,13.15,29.60}, + {0.05602,0.00,2.460,0,0.4880,7.8310,53.60,3.1992,3,193.0,17.80,392.63,4.45,50.00}, + {0.07875,45.00,3.440,0,0.4370,6.7820,41.10,3.7886,5,398.0,15.20,393.87,6.68,32.00}, + {0.12579,45.00,3.440,0,0.4370,6.5560,29.10,4.5667,5,398.0,15.20,382.84,4.56,29.80}, + {0.08370,45.00,3.440,0,0.4370,7.1850,38.90,4.5667,5,398.0,15.20,396.90,5.39,34.90}, + {0.09068,45.00,3.440,0,0.4370,6.9510,21.50,6.4798,5,398.0,15.20,377.68,5.10,37.00}, + {0.06911,45.00,3.440,0,0.4370,6.7390,30.80,6.4798,5,398.0,15.20,389.71,4.69,30.50}, + {0.08664,45.00,3.440,0,0.4370,7.1780,26.30,6.4798,5,398.0,15.20,390.49,2.87,36.40}, + {0.02187,60.00,2.930,0,0.4010,6.8000,9.90,6.2196,1,265.0,15.60,393.37,5.03,31.10}, + {0.01439,60.00,2.930,0,0.4010,6.6040,18.80,6.2196,1,265.0,15.60,376.70,4.38,29.10}, + {0.01381,80.00,0.460,0,0.4220,7.8750,32.00,5.6484,4,255.0,14.40,394.23,2.97,50.00}, + {0.04011,80.00,1.520,0,0.4040,7.2870,34.10,7.3090,2,329.0,12.60,396.90,4.08,33.30}, + {0.04666,80.00,1.520,0,0.4040,7.1070,36.60,7.3090,2,329.0,12.60,354.31,8.61,30.30}, + {0.03768,80.00,1.520,0,0.4040,7.2740,38.30,7.3090,2,329.0,12.60,392.20,6.62,34.60}, + {0.03150,95.00,1.470,0,0.4030,6.9750,15.30,7.6534,3,402.0,17.00,396.90,4.56,34.90}, + {0.01778,95.00,1.470,0,0.4030,7.1350,13.90,7.6534,3,402.0,17.00,384.30,4.45,32.90}, + {0.03445,82.50,2.030,0,0.4150,6.1620,38.40,6.2700,2,348.0,14.70,393.77,7.43,24.10}, + {0.02177,82.50,2.030,0,0.4150,7.6100,15.70,6.2700,2,348.0,14.70,395.38,3.11,42.30}, + {0.03510,95.00,2.680,0,0.4161,7.8530,33.20,5.1180,4,224.0,14.70,392.78,3.81,48.50}, + {0.02009,95.00,2.680,0,0.4161,8.0340,31.90,5.1180,4,224.0,14.70,390.55,2.88,50.00}, + {0.13642,0.00,10.590,0,0.4890,5.8910,22.30,3.9454,4,277.0,18.60,396.90,10.87,22.60}, + {0.22969,0.00,10.590,0,0.4890,6.3260,52.50,4.3549,4,277.0,18.60,394.87,10.97,24.40}, + {0.25199,0.00,10.590,0,0.4890,5.7830,72.70,4.3549,4,277.0,18.60,389.43,18.06,22.50}, + {0.13587,0.00,10.590,1,0.4890,6.0640,59.10,4.2392,4,277.0,18.60,381.32,14.66,24.40}, + {0.43571,0.00,10.590,1,0.4890,5.3440,100.00,3.8750,4,277.0,18.60,396.90,23.09,20.00}, + {0.17446,0.00,10.590,1,0.4890,5.9600,92.10,3.8771,4,277.0,18.60,393.25,17.27,21.70}, + {0.37578,0.00,10.590,1,0.4890,5.4040,88.60,3.6650,4,277.0,18.60,395.24,23.98,19.30}, + {0.21719,0.00,10.590,1,0.4890,5.8070,53.80,3.6526,4,277.0,18.60,390.94,16.03,22.40}, + {0.14052,0.00,10.590,0,0.4890,6.3750,32.30,3.9454,4,277.0,18.60,385.81,9.38,28.10}, + {0.28955,0.00,10.590,0,0.4890,5.4120,9.80,3.5875,4,277.0,18.60,348.93,29.55,23.70}, + {0.19802,0.00,10.590,0,0.4890,6.1820,42.40,3.9454,4,277.0,18.60,393.63,9.47,25.00}, + {0.04560,0.00,13.890,1,0.5500,5.8880,56.00,3.1121,5,276.0,16.40,392.80,13.51,23.30}, + {0.07013,0.00,13.890,0,0.5500,6.6420,85.10,3.4211,5,276.0,16.40,392.78,9.69,28.70}, + {0.11069,0.00,13.890,1,0.5500,5.9510,93.80,2.8893,5,276.0,16.40,396.90,17.92,21.50}, + {0.11425,0.00,13.890,1,0.5500,6.3730,92.40,3.3633,5,276.0,16.40,393.74,10.50,23.00}, + {0.35809,0.00,6.200,1,0.5070,6.9510,88.50,2.8617,8,307.0,17.40,391.70,9.71,26.70}, + {0.40771,0.00,6.200,1,0.5070,6.1640,91.30,3.0480,8,307.0,17.40,395.24,21.46,21.70}, + {0.62356,0.00,6.200,1,0.5070,6.8790,77.70,3.2721,8,307.0,17.40,390.39,9.93,27.50}, + {0.61470,0.00,6.200,0,0.5070,6.6180,80.80,3.2721,8,307.0,17.40,396.90,7.60,30.10}, + {0.31533,0.00,6.200,0,0.5040,8.2660,78.30,2.8944,8,307.0,17.40,385.05,4.14,44.80}, + {0.52693,0.00,6.200,0,0.5040,8.7250,83.00,2.8944,8,307.0,17.40,382.00,4.63,50.00}, + {0.38214,0.00,6.200,0,0.5040,8.0400,86.50,3.2157,8,307.0,17.40,387.38,3.13,37.60}, + {0.41238,0.00,6.200,0,0.5040,7.1630,79.90,3.2157,8,307.0,17.40,372.08,6.36,31.60}, + {0.29819,0.00,6.200,0,0.5040,7.6860,17.00,3.3751,8,307.0,17.40,377.51,3.92,46.70}, + {0.44178,0.00,6.200,0,0.5040,6.5520,21.40,3.3751,8,307.0,17.40,380.34,3.76,31.50}, + {0.53700,0.00,6.200,0,0.5040,5.9810,68.10,3.6715,8,307.0,17.40,378.35,11.65,24.30}, + {0.46296,0.00,6.200,0,0.5040,7.4120,76.90,3.6715,8,307.0,17.40,376.14,5.25,31.70}, + {0.57529,0.00,6.200,0,0.5070,8.3370,73.30,3.8384,8,307.0,17.40,385.91,2.47,41.70}, + {0.33147,0.00,6.200,0,0.5070,8.2470,70.40,3.6519,8,307.0,17.40,378.95,3.95,48.30}, + {0.44791,0.00,6.200,1,0.5070,6.7260,66.50,3.6519,8,307.0,17.40,360.20,8.05,29.00}, + {0.33045,0.00,6.200,0,0.5070,6.0860,61.50,3.6519,8,307.0,17.40,376.75,10.88,24.00}, + {0.52058,0.00,6.200,1,0.5070,6.6310,76.50,4.1480,8,307.0,17.40,388.45,9.54,25.10}, + {0.51183,0.00,6.200,0,0.5070,7.3580,71.60,4.1480,8,307.0,17.40,390.07,4.73,31.50}, + {0.08244,30.00,4.930,0,0.4280,6.4810,18.50,6.1899,6,300.0,16.60,379.41,6.36,23.70}, + {0.09252,30.00,4.930,0,0.4280,6.6060,42.20,6.1899,6,300.0,16.60,383.78,7.37,23.30}, + {0.11329,30.00,4.930,0,0.4280,6.8970,54.30,6.3361,6,300.0,16.60,391.25,11.38,22.00}, + {0.10612,30.00,4.930,0,0.4280,6.0950,65.10,6.3361,6,300.0,16.60,394.62,12.40,20.10}, + {0.10290,30.00,4.930,0,0.4280,6.3580,52.90,7.0355,6,300.0,16.60,372.75,11.22,22.20}, + {0.12757,30.00,4.930,0,0.4280,6.3930,7.80,7.0355,6,300.0,16.60,374.71,5.19,23.70}, + {0.20608,22.00,5.860,0,0.4310,5.5930,76.50,7.9549,7,330.0,19.10,372.49,12.50,17.60}, + {0.19133,22.00,5.860,0,0.4310,5.6050,70.20,7.9549,7,330.0,19.10,389.13,18.46,18.50}, + {0.33983,22.00,5.860,0,0.4310,6.1080,34.90,8.0555,7,330.0,19.10,390.18,9.16,24.30}, + {0.19657,22.00,5.860,0,0.4310,6.2260,79.20,8.0555,7,330.0,19.10,376.14,10.15,20.50}, + {0.16439,22.00,5.860,0,0.4310,6.4330,49.10,7.8265,7,330.0,19.10,374.71,9.52,24.50}, + {0.19073,22.00,5.860,0,0.4310,6.7180,17.50,7.8265,7,330.0,19.10,393.74,6.56,26.20}, + {0.14030,22.00,5.860,0,0.4310,6.4870,13.00,7.3967,7,330.0,19.10,396.28,5.90,24.40}, + {0.21409,22.00,5.860,0,0.4310,6.4380,8.90,7.3967,7,330.0,19.10,377.07,3.59,24.80}, + {0.08221,22.00,5.860,0,0.4310,6.9570,6.80,8.9067,7,330.0,19.10,386.09,3.53,29.60}, + {0.36894,22.00,5.860,0,0.4310,8.2590,8.40,8.9067,7,330.0,19.10,396.90,3.54,42.80}, + {0.04819,80.00,3.640,0,0.3920,6.1080,32.00,9.2203,1,315.0,16.40,392.89,6.57,21.90}, + {0.03548,80.00,3.640,0,0.3920,5.8760,19.10,9.2203,1,315.0,16.40,395.18,9.25,20.90}, + {0.01538,90.00,3.750,0,0.3940,7.4540,34.20,6.3361,3,244.0,15.90,386.34,3.11,44.00}, + {0.61154,20.00,3.970,0,0.6470,8.7040,86.90,1.8010,5,264.0,13.00,389.70,5.12,50.00}, + {0.66351,20.00,3.970,0,0.6470,7.3330,100.00,1.8946,5,264.0,13.00,383.29,7.79,36.00}, + {0.65665,20.00,3.970,0,0.6470,6.8420,100.00,2.0107,5,264.0,13.00,391.93,6.90,30.10}, + {0.54011,20.00,3.970,0,0.6470,7.2030,81.80,2.1121,5,264.0,13.00,392.80,9.59,33.80}, + {0.53412,20.00,3.970,0,0.6470,7.5200,89.40,2.1398,5,264.0,13.00,388.37,7.26,43.10}, + {0.52014,20.00,3.970,0,0.6470,8.3980,91.50,2.2885,5,264.0,13.00,386.86,5.91,48.80}, + {0.82526,20.00,3.970,0,0.6470,7.3270,94.50,2.0788,5,264.0,13.00,393.42,11.25,31.00}, + {0.55007,20.00,3.970,0,0.6470,7.2060,91.60,1.9301,5,264.0,13.00,387.89,8.10,36.50}, + {0.76162,20.00,3.970,0,0.6470,5.5600,62.80,1.9865,5,264.0,13.00,392.40,10.45,22.80}, + {0.78570,20.00,3.970,0,0.6470,7.0140,84.60,2.1329,5,264.0,13.00,384.07,14.79,30.70}, + {0.57834,20.00,3.970,0,0.5750,8.2970,67.00,2.4216,5,264.0,13.00,384.54,7.44,50.00}, + {0.54050,20.00,3.970,0,0.5750,7.4700,52.60,2.8720,5,264.0,13.00,390.30,3.16,43.50}, + {0.09065,20.00,6.960,1,0.4640,5.9200,61.50,3.9175,3,223.0,18.60,391.34,13.65,20.70}, + {0.29916,20.00,6.960,0,0.4640,5.8560,42.10,4.4290,3,223.0,18.60,388.65,13.00,21.10}, + {0.16211,20.00,6.960,0,0.4640,6.2400,16.30,4.4290,3,223.0,18.60,396.90,6.59,25.20}, + {0.11460,20.00,6.960,0,0.4640,6.5380,58.70,3.9175,3,223.0,18.60,394.96,7.73,24.40}, + {0.22188,20.00,6.960,1,0.4640,7.6910,51.80,4.3665,3,223.0,18.60,390.77,6.58,35.20}, + {0.05644,40.00,6.410,1,0.4470,6.7580,32.90,4.0776,4,254.0,17.60,396.90,3.53,32.40}, + {0.09604,40.00,6.410,0,0.4470,6.8540,42.80,4.2673,4,254.0,17.60,396.90,2.98,32.00}, + {0.10469,40.00,6.410,1,0.4470,7.2670,49.00,4.7872,4,254.0,17.60,389.25,6.05,33.20}, + {0.06127,40.00,6.410,1,0.4470,6.8260,27.60,4.8628,4,254.0,17.60,393.45,4.16,33.10}, + {0.07978,40.00,6.410,0,0.4470,6.4820,32.10,4.1403,4,254.0,17.60,396.90,7.19,29.10}, + {0.21038,20.00,3.330,0,0.4429,6.8120,32.20,4.1007,5,216.0,14.90,396.90,4.85,35.10}, + {0.03578,20.00,3.330,0,0.4429,7.8200,64.50,4.6947,5,216.0,14.90,387.31,3.76,45.40}, + {0.03705,20.00,3.330,0,0.4429,6.9680,37.20,5.2447,5,216.0,14.90,392.23,4.59,35.40}, + {0.06129,20.00,3.330,1,0.4429,7.6450,49.70,5.2119,5,216.0,14.90,377.07,3.01,46.00}, + {0.01501,90.00,1.210,1,0.4010,7.9230,24.80,5.8850,1,198.0,13.60,395.52,3.16,50.00}, + {0.00906,90.00,2.970,0,0.4000,7.0880,20.80,7.3073,1,285.0,15.30,394.72,7.85,32.20}, + {0.01096,55.00,2.250,0,0.3890,6.4530,31.90,7.3073,1,300.0,15.30,394.72,8.23,22.00}, + {0.01965,80.00,1.760,0,0.3850,6.2300,31.50,9.0892,1,241.0,18.20,341.60,12.93,20.10}, + {0.03871,52.50,5.320,0,0.4050,6.2090,31.30,7.3172,6,293.0,16.60,396.90,7.14,23.20}, + {0.04590,52.50,5.320,0,0.4050,6.3150,45.60,7.3172,6,293.0,16.60,396.90,7.60,22.30}, + {0.04297,52.50,5.320,0,0.4050,6.5650,22.90,7.3172,6,293.0,16.60,371.72,9.51,24.80}, + {0.03502,80.00,4.950,0,0.4110,6.8610,27.90,5.1167,4,245.0,19.20,396.90,3.33,28.50}, + {0.07886,80.00,4.950,0,0.4110,7.1480,27.70,5.1167,4,245.0,19.20,396.90,3.56,37.30}, + {0.03615,80.00,4.950,0,0.4110,6.6300,23.40,5.1167,4,245.0,19.20,396.90,4.70,27.90}, + {0.08265,0.00,13.920,0,0.4370,6.1270,18.40,5.5027,4,289.0,16.00,396.90,8.58,23.90}, + {0.08199,0.00,13.920,0,0.4370,6.0090,42.30,5.5027,4,289.0,16.00,396.90,10.40,21.70}, + {0.12932,0.00,13.920,0,0.4370,6.6780,31.10,5.9604,4,289.0,16.00,396.90,6.27,28.60}, + {0.05372,0.00,13.920,0,0.4370,6.5490,51.00,5.9604,4,289.0,16.00,392.85,7.39,27.10}, + {0.14103,0.00,13.920,0,0.4370,5.7900,58.00,6.3200,4,289.0,16.00,396.90,15.84,20.30}, + {0.06466,70.00,2.240,0,0.4000,6.3450,20.10,7.8278,5,358.0,14.80,368.24,4.97,22.50}, + {0.05561,70.00,2.240,0,0.4000,7.0410,10.00,7.8278,5,358.0,14.80,371.58,4.74,29.00}, + {0.04417,70.00,2.240,0,0.4000,6.8710,47.40,7.8278,5,358.0,14.80,390.86,6.07,24.80}, + {0.03537,34.00,6.090,0,0.4330,6.5900,40.40,5.4917,7,329.0,16.10,395.75,9.50,22.00}, + {0.09266,34.00,6.090,0,0.4330,6.4950,18.40,5.4917,7,329.0,16.10,383.61,8.67,26.40}, + {0.10000,34.00,6.090,0,0.4330,6.9820,17.70,5.4917,7,329.0,16.10,390.43,4.86,33.10}, + {0.05515,33.00,2.180,0,0.4720,7.2360,41.10,4.0220,7,222.0,18.40,393.68,6.93,36.10}, + {0.05479,33.00,2.180,0,0.4720,6.6160,58.10,3.3700,7,222.0,18.40,393.36,8.93,28.40}, + {0.07503,33.00,2.180,0,0.4720,7.4200,71.90,3.0992,7,222.0,18.40,396.90,6.47,33.40}, + {0.04932,33.00,2.180,0,0.4720,6.8490,70.30,3.1827,7,222.0,18.40,396.90,7.53,28.20}, + {0.49298,0.00,9.900,0,0.5440,6.6350,82.50,3.3175,4,304.0,18.40,396.90,4.54,22.80}, + {0.34940,0.00,9.900,0,0.5440,5.9720,76.70,3.1025,4,304.0,18.40,396.24,9.97,20.30}, + {2.63548,0.00,9.900,0,0.5440,4.9730,37.80,2.5194,4,304.0,18.40,350.45,12.64,16.10}, + {0.79041,0.00,9.900,0,0.5440,6.1220,52.80,2.6403,4,304.0,18.40,396.90,5.98,22.10}, + {0.26169,0.00,9.900,0,0.5440,6.0230,90.40,2.8340,4,304.0,18.40,396.30,11.72,19.40}, + {0.26938,0.00,9.900,0,0.5440,6.2660,82.80,3.2628,4,304.0,18.40,393.39,7.90,21.60}, + {0.36920,0.00,9.900,0,0.5440,6.5670,87.30,3.6023,4,304.0,18.40,395.69,9.28,23.80}, + {0.25356,0.00,9.900,0,0.5440,5.7050,77.70,3.9450,4,304.0,18.40,396.42,11.50,16.20}, + {0.31827,0.00,9.900,0,0.5440,5.9140,83.20,3.9986,4,304.0,18.40,390.70,18.33,17.80}, + {0.24522,0.00,9.900,0,0.5440,5.7820,71.70,4.0317,4,304.0,18.40,396.90,15.94,19.80}, + {0.40202,0.00,9.900,0,0.5440,6.3820,67.20,3.5325,4,304.0,18.40,395.21,10.36,23.10}, + {0.47547,0.00,9.900,0,0.5440,6.1130,58.80,4.0019,4,304.0,18.40,396.23,12.73,21.00}, + {0.16760,0.00,7.380,0,0.4930,6.4260,52.30,4.5404,5,287.0,19.60,396.90,7.20,23.80}, + {0.18159,0.00,7.380,0,0.4930,6.3760,54.30,4.5404,5,287.0,19.60,396.90,6.87,23.10}, + {0.35114,0.00,7.380,0,0.4930,6.0410,49.90,4.7211,5,287.0,19.60,396.90,7.70,20.40}, + {0.28392,0.00,7.380,0,0.4930,5.7080,74.30,4.7211,5,287.0,19.60,391.13,11.74,18.50}, + {0.34109,0.00,7.380,0,0.4930,6.4150,40.10,4.7211,5,287.0,19.60,396.90,6.12,25.00}, + {0.19186,0.00,7.380,0,0.4930,6.4310,14.70,5.4159,5,287.0,19.60,393.68,5.08,24.60}, + {0.30347,0.00,7.380,0,0.4930,6.3120,28.90,5.4159,5,287.0,19.60,396.90,6.15,23.00}, + {0.24103,0.00,7.380,0,0.4930,6.0830,43.70,5.4159,5,287.0,19.60,396.90,12.79,22.20}, + {0.06617,0.00,3.240,0,0.4600,5.8680,25.80,5.2146,4,430.0,16.90,382.44,9.97,19.30}, + {0.06724,0.00,3.240,0,0.4600,6.3330,17.20,5.2146,4,430.0,16.90,375.21,7.34,22.60}, + {0.04544,0.00,3.240,0,0.4600,6.1440,32.20,5.8736,4,430.0,16.90,368.57,9.09,19.80}, + {0.05023,35.00,6.060,0,0.4379,5.7060,28.40,6.6407,1,304.0,16.90,394.02,12.43,17.10}, + {0.03466,35.00,6.060,0,0.4379,6.0310,23.30,6.6407,1,304.0,16.90,362.25,7.83,19.40}, + {0.05083,0.00,5.190,0,0.5150,6.3160,38.10,6.4584,5,224.0,20.20,389.71,5.68,22.20}, + {0.03738,0.00,5.190,0,0.5150,6.3100,38.50,6.4584,5,224.0,20.20,389.40,6.75,20.70}, + {0.03961,0.00,5.190,0,0.5150,6.0370,34.50,5.9853,5,224.0,20.20,396.90,8.01,21.10}, + {0.03427,0.00,5.190,0,0.5150,5.8690,46.30,5.2311,5,224.0,20.20,396.90,9.80,19.50}, + {0.03041,0.00,5.190,0,0.5150,5.8950,59.60,5.6150,5,224.0,20.20,394.81,10.56,18.50}, + {0.03306,0.00,5.190,0,0.5150,6.0590,37.30,4.8122,5,224.0,20.20,396.14,8.51,20.60}, + {0.05497,0.00,5.190,0,0.5150,5.9850,45.40,4.8122,5,224.0,20.20,396.90,9.74,19.00}, + {0.06151,0.00,5.190,0,0.5150,5.9680,58.50,4.8122,5,224.0,20.20,396.90,9.29,18.70}, + {0.01301,35.00,1.520,0,0.4420,7.2410,49.30,7.0379,1,284.0,15.50,394.74,5.49,32.70}, + {0.02498,0.00,1.890,0,0.5180,6.5400,59.70,6.2669,1,422.0,15.90,389.96,8.65,16.50}, + {0.02543,55.00,3.780,0,0.4840,6.6960,56.40,5.7321,5,370.0,17.60,396.90,7.18,23.90}, + {0.03049,55.00,3.780,0,0.4840,6.8740,28.10,6.4654,5,370.0,17.60,387.97,4.61,31.20}, + {0.03113,0.00,4.390,0,0.4420,6.0140,48.50,8.0136,3,352.0,18.80,385.64,10.53,17.50}, + {0.06162,0.00,4.390,0,0.4420,5.8980,52.30,8.0136,3,352.0,18.80,364.61,12.67,17.20}, + {0.01870,85.00,4.150,0,0.4290,6.5160,27.70,8.5353,4,351.0,17.90,392.43,6.36,23.10}, + {0.01501,80.00,2.010,0,0.4350,6.6350,29.70,8.3440,4,280.0,17.00,390.94,5.99,24.50}, + {0.02899,40.00,1.250,0,0.4290,6.9390,34.50,8.7921,1,335.0,19.70,389.85,5.89,26.60}, + {0.06211,40.00,1.250,0,0.4290,6.4900,44.40,8.7921,1,335.0,19.70,396.90,5.98,22.90}, + {0.07950,60.00,1.690,0,0.4110,6.5790,35.90,10.7103,4,411.0,18.30,370.78,5.49,24.10}, + {0.07244,60.00,1.690,0,0.4110,5.8840,18.50,10.7103,4,411.0,18.30,392.33,7.79,18.60}, + {0.01709,90.00,2.020,0,0.4100,6.7280,36.10,12.1265,5,187.0,17.00,384.46,4.50,30.10}, + {0.04301,80.00,1.910,0,0.4130,5.6630,21.90,10.5857,4,334.0,22.00,382.80,8.05,18.20}, + {0.10659,80.00,1.910,0,0.4130,5.9360,19.50,10.5857,4,334.0,22.00,376.04,5.57,20.60}, + {8.98296,0.00,18.100,1,0.7700,6.2120,97.40,2.1222,24,666.0,20.20,377.73,17.60,17.80}, + {3.84970,0.00,18.100,1,0.7700,6.3950,91.00,2.5052,24,666.0,20.20,391.34,13.27,21.70}, + {5.20177,0.00,18.100,1,0.7700,6.1270,83.40,2.7227,24,666.0,20.20,395.43,11.48,22.70}, + {4.26131,0.00,18.100,0,0.7700,6.1120,81.30,2.5091,24,666.0,20.20,390.74,12.67,22.60}, + {4.54192,0.00,18.100,0,0.7700,6.3980,88.00,2.5182,24,666.0,20.20,374.56,7.79,25.00}, + {3.83684,0.00,18.100,0,0.7700,6.2510,91.10,2.2955,24,666.0,20.20,350.65,14.19,19.90}, + {3.67822,0.00,18.100,0,0.7700,5.3620,96.20,2.1036,24,666.0,20.20,380.79,10.19,20.80}, + {4.22239,0.00,18.100,1,0.7700,5.8030,89.00,1.9047,24,666.0,20.20,353.04,14.64,16.80}, + {3.47428,0.00,18.100,1,0.7180,8.7800,82.90,1.9047,24,666.0,20.20,354.55,5.29,21.90}, + {4.55587,0.00,18.100,0,0.7180,3.5610,87.90,1.6132,24,666.0,20.20,354.70,7.12,27.50}, + {3.69695,0.00,18.100,0,0.7180,4.9630,91.40,1.7523,24,666.0,20.20,316.03,14.00,21.90}, + {13.52220,0.00,18.100,0,0.6310,3.8630,100.00,1.5106,24,666.0,20.20,131.42,13.33,23.10}, + {4.89822,0.00,18.100,0,0.6310,4.9700,100.00,1.3325,24,666.0,20.20,375.52,3.26,50.00}, + {5.66998,0.00,18.100,1,0.6310,6.6830,96.80,1.3567,24,666.0,20.20,375.33,3.73,50.00}, + {6.53876,0.00,18.100,1,0.6310,7.0160,97.50,1.2024,24,666.0,20.20,392.05,2.96,50.00}, + {9.23230,0.00,18.100,0,0.6310,6.2160,100.00,1.1691,24,666.0,20.20,366.15,9.53,50.00}, + {8.26725,0.00,18.100,1,0.6680,5.8750,89.60,1.1296,24,666.0,20.20,347.88,8.88,50.00}, + {11.10810,0.00,18.100,0,0.6680,4.9060,100.00,1.1742,24,666.0,20.20,396.90,34.77,13.80}, + {18.49820,0.00,18.100,0,0.6680,4.1380,100.00,1.1370,24,666.0,20.20,396.90,37.97,13.80}, + {19.60910,0.00,18.100,0,0.6710,7.3130,97.90,1.3163,24,666.0,20.20,396.90,13.44,15.00}, + {15.28800,0.00,18.100,0,0.6710,6.6490,93.30,1.3449,24,666.0,20.20,363.02,23.24,13.90}, + {9.82349,0.00,18.100,0,0.6710,6.7940,98.80,1.3580,24,666.0,20.20,396.90,21.24,13.30}, + {23.64820,0.00,18.100,0,0.6710,6.3800,96.20,1.3861,24,666.0,20.20,396.90,23.69,13.10}, + {17.86670,0.00,18.100,0,0.6710,6.2230,100.00,1.3861,24,666.0,20.20,393.74,21.78,10.20}, + {88.97620,0.00,18.100,0,0.6710,6.9680,91.90,1.4165,24,666.0,20.20,396.90,17.21,10.40}, + {15.87440,0.00,18.100,0,0.6710,6.5450,99.10,1.5192,24,666.0,20.20,396.90,21.08,10.90}, + {9.18702,0.00,18.100,0,0.7000,5.5360,100.00,1.5804,24,666.0,20.20,396.90,23.60,11.30}, + {7.99248,0.00,18.100,0,0.7000,5.5200,100.00,1.5331,24,666.0,20.20,396.90,24.56,12.30}, + {20.08490,0.00,18.100,0,0.7000,4.3680,91.20,1.4395,24,666.0,20.20,285.83,30.63,8.80}, + {16.81180,0.00,18.100,0,0.7000,5.2770,98.10,1.4261,24,666.0,20.20,396.90,30.81,7.20}, + {24.39380,0.00,18.100,0,0.7000,4.6520,100.00,1.4672,24,666.0,20.20,396.90,28.28,10.50}, + {22.59710,0.00,18.100,0,0.7000,5.0000,89.50,1.5184,24,666.0,20.20,396.90,31.99,7.40}, + {14.33370,0.00,18.100,0,0.7000,4.8800,100.00,1.5895,24,666.0,20.20,372.92,30.62,10.20}, + {8.15174,0.00,18.100,0,0.7000,5.3900,98.90,1.7281,24,666.0,20.20,396.90,20.85,11.50}, + {6.96215,0.00,18.100,0,0.7000,5.7130,97.00,1.9265,24,666.0,20.20,394.43,17.11,15.10}, + {5.29305,0.00,18.100,0,0.7000,6.0510,82.50,2.1678,24,666.0,20.20,378.38,18.76,23.20}, + {11.57790,0.00,18.100,0,0.7000,5.0360,97.00,1.7700,24,666.0,20.20,396.90,25.68,9.70}, + {8.64476,0.00,18.100,0,0.6930,6.1930,92.60,1.7912,24,666.0,20.20,396.90,15.17,13.80}, + {13.35980,0.00,18.100,0,0.6930,5.8870,94.70,1.7821,24,666.0,20.20,396.90,16.35,12.70}, + {8.71675,0.00,18.100,0,0.6930,6.4710,98.80,1.7257,24,666.0,20.20,391.98,17.12,13.10}, + {5.87205,0.00,18.100,0,0.6930,6.4050,96.00,1.6768,24,666.0,20.20,396.90,19.37,12.50}, + {7.67202,0.00,18.100,0,0.6930,5.7470,98.90,1.6334,24,666.0,20.20,393.10,19.92,8.50}, + {38.35180,0.00,18.100,0,0.6930,5.4530,100.00,1.4896,24,666.0,20.20,396.90,30.59,5.00}, + {9.91655,0.00,18.100,0,0.6930,5.8520,77.80,1.5004,24,666.0,20.20,338.16,29.97,6.30}, + {25.04610,0.00,18.100,0,0.6930,5.9870,100.00,1.5888,24,666.0,20.20,396.90,26.77,5.60}, + {14.23620,0.00,18.100,0,0.6930,6.3430,100.00,1.5741,24,666.0,20.20,396.90,20.32,7.20}, + {9.59571,0.00,18.100,0,0.6930,6.4040,100.00,1.6390,24,666.0,20.20,376.11,20.31,12.10}, + {24.80170,0.00,18.100,0,0.6930,5.3490,96.00,1.7028,24,666.0,20.20,396.90,19.77,8.30}, + {41.52920,0.00,18.100,0,0.6930,5.5310,85.40,1.6074,24,666.0,20.20,329.46,27.38,8.50}, + {67.92080,0.00,18.100,0,0.6930,5.6830,100.00,1.4254,24,666.0,20.20,384.97,22.98,5.00}, + {20.71620,0.00,18.100,0,0.6590,4.1380,100.00,1.1781,24,666.0,20.20,370.22,23.34,11.90}, + {11.95110,0.00,18.100,0,0.6590,5.6080,100.00,1.2852,24,666.0,20.20,332.09,12.13,27.90}, + {7.40389,0.00,18.100,0,0.5970,5.6170,97.90,1.4547,24,666.0,20.20,314.64,26.40,17.20}, + {14.43830,0.00,18.100,0,0.5970,6.8520,100.00,1.4655,24,666.0,20.20,179.36,19.78,27.50}, + {51.13580,0.00,18.100,0,0.5970,5.7570,100.00,1.4130,24,666.0,20.20,2.60,10.11,15.00}, + {14.05070,0.00,18.100,0,0.5970,6.6570,100.00,1.5275,24,666.0,20.20,35.05,21.22,17.20}, + {18.81100,0.00,18.100,0,0.5970,4.6280,100.00,1.5539,24,666.0,20.20,28.79,34.37,17.90}, + {28.65580,0.00,18.100,0,0.5970,5.1550,100.00,1.5894,24,666.0,20.20,210.97,20.08,16.30}, + {45.74610,0.00,18.100,0,0.6930,4.5190,100.00,1.6582,24,666.0,20.20,88.27,36.98,7.00}, + {18.08460,0.00,18.100,0,0.6790,6.4340,100.00,1.8347,24,666.0,20.20,27.25,29.05,7.20}, + {10.83420,0.00,18.100,0,0.6790,6.7820,90.80,1.8195,24,666.0,20.20,21.57,25.79,7.50}, + {25.94060,0.00,18.100,0,0.6790,5.3040,89.10,1.6475,24,666.0,20.20,127.36,26.64,10.40}, + {73.53410,0.00,18.100,0,0.6790,5.9570,100.00,1.8026,24,666.0,20.20,16.45,20.62,8.80}, + {11.81230,0.00,18.100,0,0.7180,6.8240,76.50,1.7940,24,666.0,20.20,48.45,22.74,8.40}, + {11.08740,0.00,18.100,0,0.7180,6.4110,100.00,1.8589,24,666.0,20.20,318.75,15.02,16.70}, + {7.02259,0.00,18.100,0,0.7180,6.0060,95.30,1.8746,24,666.0,20.20,319.98,15.70,14.20}, + {12.04820,0.00,18.100,0,0.6140,5.6480,87.60,1.9512,24,666.0,20.20,291.55,14.10,20.80}, + {7.05042,0.00,18.100,0,0.6140,6.1030,85.10,2.0218,24,666.0,20.20,2.52,23.29,13.40}, + {8.79212,0.00,18.100,0,0.5840,5.5650,70.60,2.0635,24,666.0,20.20,3.65,17.16,11.70}, + {15.86030,0.00,18.100,0,0.6790,5.8960,95.40,1.9096,24,666.0,20.20,7.68,24.39,8.30}, + {12.24720,0.00,18.100,0,0.5840,5.8370,59.70,1.9976,24,666.0,20.20,24.65,15.69,10.20}, + {37.66190,0.00,18.100,0,0.6790,6.2020,78.70,1.8629,24,666.0,20.20,18.82,14.52,10.90}, + {7.36711,0.00,18.100,0,0.6790,6.1930,78.10,1.9356,24,666.0,20.20,96.73,21.52,11.00}, + {9.33889,0.00,18.100,0,0.6790,6.3800,95.60,1.9682,24,666.0,20.20,60.72,24.08,9.50}, + {8.49213,0.00,18.100,0,0.5840,6.3480,86.10,2.0527,24,666.0,20.20,83.45,17.64,14.50}, + {10.06230,0.00,18.100,0,0.5840,6.8330,94.30,2.0882,24,666.0,20.20,81.33,19.69,14.10}, + {6.44405,0.00,18.100,0,0.5840,6.4250,74.80,2.2004,24,666.0,20.20,97.95,12.03,16.10}, + {5.58107,0.00,18.100,0,0.7130,6.4360,87.90,2.3158,24,666.0,20.20,100.19,16.22,14.30}, + {13.91340,0.00,18.100,0,0.7130,6.2080,95.00,2.2222,24,666.0,20.20,100.63,15.17,11.70}, + {11.16040,0.00,18.100,0,0.7400,6.6290,94.60,2.1247,24,666.0,20.20,109.85,23.27,13.40}, + {14.42080,0.00,18.100,0,0.7400,6.4610,93.30,2.0026,24,666.0,20.20,27.49,18.05,9.60}, + {15.17720,0.00,18.100,0,0.7400,6.1520,100.00,1.9142,24,666.0,20.20,9.32,26.45,8.70}, + {13.67810,0.00,18.100,0,0.7400,5.9350,87.90,1.8206,24,666.0,20.20,68.95,34.02,8.40}, + {9.39063,0.00,18.100,0,0.7400,5.6270,93.90,1.8172,24,666.0,20.20,396.90,22.88,12.80}, + {22.05110,0.00,18.100,0,0.7400,5.8180,92.40,1.8662,24,666.0,20.20,391.45,22.11,10.50}, + {9.72418,0.00,18.100,0,0.7400,6.4060,97.20,2.0651,24,666.0,20.20,385.96,19.52,17.10}, + {5.66637,0.00,18.100,0,0.7400,6.2190,100.00,2.0048,24,666.0,20.20,395.69,16.59,18.40}, + {9.96654,0.00,18.100,0,0.7400,6.4850,100.00,1.9784,24,666.0,20.20,386.73,18.85,15.40}, + {12.80230,0.00,18.100,0,0.7400,5.8540,96.60,1.8956,24,666.0,20.20,240.52,23.79,10.80}, + {10.67180,0.00,18.100,0,0.7400,6.4590,94.80,1.9879,24,666.0,20.20,43.06,23.98,11.80}, + {6.28807,0.00,18.100,0,0.7400,6.3410,96.40,2.0720,24,666.0,20.20,318.01,17.79,14.90}, + {9.92485,0.00,18.100,0,0.7400,6.2510,96.60,2.1980,24,666.0,20.20,388.52,16.44,12.60}, + {9.32909,0.00,18.100,0,0.7130,6.1850,98.70,2.2616,24,666.0,20.20,396.90,18.13,14.10}, + {7.52601,0.00,18.100,0,0.7130,6.4170,98.30,2.1850,24,666.0,20.20,304.21,19.31,13.00}, + {6.71772,0.00,18.100,0,0.7130,6.7490,92.60,2.3236,24,666.0,20.20,0.32,17.44,13.40}, + {5.44114,0.00,18.100,0,0.7130,6.6550,98.20,2.3552,24,666.0,20.20,355.29,17.73,15.20}, + {5.09017,0.00,18.100,0,0.7130,6.2970,91.80,2.3682,24,666.0,20.20,385.09,17.27,16.10}, + {8.24809,0.00,18.100,0,0.7130,7.3930,99.30,2.4527,24,666.0,20.20,375.87,16.74,17.80}, + {9.51363,0.00,18.100,0,0.7130,6.7280,94.10,2.4961,24,666.0,20.20,6.68,18.71,14.90}, + {4.75237,0.00,18.100,0,0.7130,6.5250,86.50,2.4358,24,666.0,20.20,50.92,18.13,14.10}, + {4.66883,0.00,18.100,0,0.7130,5.9760,87.90,2.5806,24,666.0,20.20,10.48,19.01,12.70}, + {8.20058,0.00,18.100,0,0.7130,5.9360,80.30,2.7792,24,666.0,20.20,3.50,16.94,13.50}, + {7.75223,0.00,18.100,0,0.7130,6.3010,83.70,2.7831,24,666.0,20.20,272.21,16.23,14.90}, + {6.80117,0.00,18.100,0,0.7130,6.0810,84.40,2.7175,24,666.0,20.20,396.90,14.70,20.00}, + {4.81213,0.00,18.100,0,0.7130,6.7010,90.00,2.5975,24,666.0,20.20,255.23,16.42,16.40}, + {3.69311,0.00,18.100,0,0.7130,6.3760,88.40,2.5671,24,666.0,20.20,391.43,14.65,17.70}, + {6.65492,0.00,18.100,0,0.7130,6.3170,83.00,2.7344,24,666.0,20.20,396.90,13.99,19.50}, + {5.82115,0.00,18.100,0,0.7130,6.5130,89.90,2.8016,24,666.0,20.20,393.82,10.29,20.20}, + {7.83932,0.00,18.100,0,0.6550,6.2090,65.40,2.9634,24,666.0,20.20,396.90,13.22,21.40}, + {3.16360,0.00,18.100,0,0.6550,5.7590,48.20,3.0665,24,666.0,20.20,334.40,14.13,19.90}, + {3.77498,0.00,18.100,0,0.6550,5.9520,84.70,2.8715,24,666.0,20.20,22.01,17.15,19.00}, + {4.42228,0.00,18.100,0,0.5840,6.0030,94.50,2.5403,24,666.0,20.20,331.29,21.32,19.10}, + {15.57570,0.00,18.100,0,0.5800,5.9260,71.00,2.9084,24,666.0,20.20,368.74,18.13,19.10}, + {13.07510,0.00,18.100,0,0.5800,5.7130,56.70,2.8237,24,666.0,20.20,396.90,14.76,20.10}, + {4.34879,0.00,18.100,0,0.5800,6.1670,84.00,3.0334,24,666.0,20.20,396.90,16.29,19.90}, + {4.03841,0.00,18.100,0,0.5320,6.2290,90.70,3.0993,24,666.0,20.20,395.33,12.87,19.60}, + {3.56868,0.00,18.100,0,0.5800,6.4370,75.00,2.8965,24,666.0,20.20,393.37,14.36,23.20}, + {4.64689,0.00,18.100,0,0.6140,6.9800,67.60,2.5329,24,666.0,20.20,374.68,11.66,29.80}, + {8.05579,0.00,18.100,0,0.5840,5.4270,95.40,2.4298,24,666.0,20.20,352.58,18.14,13.80}, + {6.39312,0.00,18.100,0,0.5840,6.1620,97.40,2.2060,24,666.0,20.20,302.76,24.10,13.30}, + {4.87141,0.00,18.100,0,0.6140,6.4840,93.60,2.3053,24,666.0,20.20,396.21,18.68,16.70}, + {15.02340,0.00,18.100,0,0.6140,5.3040,97.30,2.1007,24,666.0,20.20,349.48,24.91,12.00}, + {10.23300,0.00,18.100,0,0.6140,6.1850,96.70,2.1705,24,666.0,20.20,379.70,18.03,14.60}, + {14.33370,0.00,18.100,0,0.6140,6.2290,88.00,1.9512,24,666.0,20.20,383.32,13.11,21.40}, + {5.82401,0.00,18.100,0,0.5320,6.2420,64.70,3.4242,24,666.0,20.20,396.90,10.74,23.00}, + {5.70818,0.00,18.100,0,0.5320,6.7500,74.90,3.3317,24,666.0,20.20,393.07,7.74,23.70}, + {5.73116,0.00,18.100,0,0.5320,7.0610,77.00,3.4106,24,666.0,20.20,395.28,7.01,25.00}, + {2.81838,0.00,18.100,0,0.5320,5.7620,40.30,4.0983,24,666.0,20.20,392.92,10.42,21.80}, + {2.37857,0.00,18.100,0,0.5830,5.8710,41.90,3.7240,24,666.0,20.20,370.73,13.34,20.60}, + {3.67367,0.00,18.100,0,0.5830,6.3120,51.90,3.9917,24,666.0,20.20,388.62,10.58,21.20}, + {5.69175,0.00,18.100,0,0.5830,6.1140,79.80,3.5459,24,666.0,20.20,392.68,14.98,19.10}, + {4.83567,0.00,18.100,0,0.5830,5.9050,53.20,3.1523,24,666.0,20.20,388.22,11.45,20.60}, + {0.15086,0.00,27.740,0,0.6090,5.4540,92.70,1.8209,4,711.0,20.10,395.09,18.06,15.20}, + {0.18337,0.00,27.740,0,0.6090,5.4140,98.30,1.7554,4,711.0,20.10,344.05,23.97,7.00}, + {0.20746,0.00,27.740,0,0.6090,5.0930,98.00,1.8226,4,711.0,20.10,318.43,29.68,8.10}, + {0.10574,0.00,27.740,0,0.6090,5.9830,98.80,1.8681,4,711.0,20.10,390.11,18.07,13.60}, + {0.11132,0.00,27.740,0,0.6090,5.9830,83.50,2.1099,4,711.0,20.10,396.90,13.35,20.10}, + {0.17331,0.00,9.690,0,0.5850,5.7070,54.00,2.3817,6,391.0,19.20,396.90,12.01,21.80}, + {0.27957,0.00,9.690,0,0.5850,5.9260,42.60,2.3817,6,391.0,19.20,396.90,13.59,24.50}, + {0.17899,0.00,9.690,0,0.5850,5.6700,28.80,2.7986,6,391.0,19.20,393.29,17.60,23.10}, + {0.28960,0.00,9.690,0,0.5850,5.3900,72.90,2.7986,6,391.0,19.20,396.90,21.14,19.70}, + {0.26838,0.00,9.690,0,0.5850,5.7940,70.60,2.8927,6,391.0,19.20,396.90,14.10,18.30}, + {0.23912,0.00,9.690,0,0.5850,6.0190,65.30,2.4091,6,391.0,19.20,396.90,12.92,21.20}, + {0.17783,0.00,9.690,0,0.5850,5.5690,73.50,2.3999,6,391.0,19.20,395.77,15.10,17.50}, + {0.22438,0.00,9.690,0,0.5850,6.0270,79.70,2.4982,6,391.0,19.20,396.90,14.33,16.80}, + {0.06263,0.00,11.930,0,0.5730,6.5930,69.10,2.4786,1,273.0,21.00,391.99,9.67,22.40}, + {0.04527,0.00,11.930,0,0.5730,6.1200,76.70,2.2875,1,273.0,21.00,396.90,9.08,20.60}, + {0.06076,0.00,11.930,0,0.5730,6.9760,91.00,2.1675,1,273.0,21.00,396.90,5.64,23.90}, + {0.10959,0.00,11.930,0,0.5730,6.7940,89.30,2.3889,1,273.0,21.00,393.45,6.48,22.00}, + {0.04741,0.00,11.930,0,0.5730,6.0300,80.80,2.5050,1,273.0,21.00,396.90,7.88,11.90} }; } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java index 636c60372fcf4..ae9f4904670ff 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/environment/EnvironmentTestSuite.java @@ -26,6 +26,7 @@ @RunWith(Suite.class) @Suite.SuiteClasses({ LearningEnvironmentBuilderTest.class, + LearningEnvironmentTest.class, PromiseTest.class }) public class EnvironmentTestSuite { diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java new file mode 100644 index 0000000000000..f88fd3e6c2404 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java @@ -0,0 +1,201 @@ +/* + * 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.ml.environment; + +import java.util.Arrays; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +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.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.environment.logging.ConsoleLogger; +import org.apache.ignite.ml.environment.logging.MLLogger; +import org.apache.ignite.ml.environment.parallelism.ParallelismStrategy; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.apache.ignite.ml.tree.randomforest.RandomForestRegressionTrainer; +import org.apache.ignite.ml.tree.randomforest.data.FeaturesCountSelectionStrategies; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.thread.IgniteThread; + +/** + * Tests for {@link LearningEnvironment} that require to start the whole Ignite infrastructure. IMPL NOTE based on + * RandomForestRegressionExample example. + */ +public class LearningEnvironmentTest extends GridCommonAbstractTest { + /** Number of nodes in grid */ + private static final int NODE_COUNT = 1; + + /** Ignite instance. */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + for (int i = 1; i <= NODE_COUNT; i++) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() { + stopAllGrids(); + } + + /** + * {@inheritDoc} + */ + @Override protected void beforeTest() { + /* Grid instance. */ + ignite = grid(NODE_COUNT); + ignite.configuration().setPeerClassLoadingEnabled(true); + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + } + + /** */ + public void testBasic() throws InterruptedException { + AtomicReference actualAmount = new AtomicReference<>(null); + AtomicReference actualMse = new AtomicReference<>(null); + AtomicReference actualMae = new AtomicReference<>(null); + + IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), + LearningEnvironmentTest.class.getSimpleName(), () -> { + IgniteCache dataCache = getTestCache(ignite); + + AtomicInteger idx = new AtomicInteger(0); + RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer( + IntStream.range(0, data[0].length - 1).mapToObj( + x -> new FeatureMeta("", idx.getAndIncrement(), false)).collect(Collectors.toList()) + ).withCountOfTrees(101) + .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) + .withMaxDepth(4) + .withMinImpurityDelta(0.) + .withSubsampleSize(0.3) + .withSeed(0); + + trainer.setEnvironment(LearningEnvironment.builder() + .withParallelismStrategy(ParallelismStrategy.Type.ON_DEFAULT_POOL) + .withLoggingFactory(ConsoleLogger.factory(MLLogger.VerboseLevel.LOW)) + .build() + ); + + ModelsComposition randomForest = trainer.fit(ignite, dataCache, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[v.length - 1] + ); + + double mse = 0.0; + double mae = 0.0; + int totalAmount = 0; + + try (QueryCursor> observations = dataCache.query(new ScanQuery<>())) { + for (Cache.Entry observation : observations) { + double difference = estimatePrediction(randomForest, observation); + + mse += Math.pow(difference, 2.0); + mae += Math.abs(difference); + + totalAmount++; + } + } + + actualAmount.set(totalAmount); + + mse = mse / totalAmount; + actualMse.set(mse); + + mae = mae / totalAmount; + actualMae.set(mae); + }); + + igniteThread.start(); + igniteThread.join(); + + assertEquals("Total amount", 23, (int)actualAmount.get()); + assertTrue("Mean squared error (MSE)", actualMse.get() > 0); + assertTrue("Mean absolute error (MAE)", actualMae.get() > 0); + } + + /** */ + private double estimatePrediction(ModelsComposition randomForest, Cache.Entry observation) { + double[] val = observation.getValue(); + double[] inputs = Arrays.copyOfRange(val, 0, val.length - 1); + double groundTruth = val[val.length - 1]; + + double prediction = randomForest.apply(VectorUtils.of(inputs)); + + return prediction - groundTruth; + } + + /** + * Fills cache with data and returns it. + * + * @param ignite Ignite instance. + * @return Filled Ignite Cache. + */ + private IgniteCache getTestCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName(UUID.randomUUID().toString()); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10)); + + IgniteCache cache = ignite.createCache(cacheConfiguration); + + for (int i = 0; i < data.length; i++) + cache.put(i, data[i]); + + return cache; + } + + /** + * Part of the Boston housing dataset. + */ + private static final double[][] data = { + {0.02731,0.00,7.070,0,0.4690,6.4210,78.90,4.9671,2,242.0,17.80,396.90,9.14,21.60}, + {0.02729,0.00,7.070,0,0.4690,7.1850,61.10,4.9671,2,242.0,17.80,392.83,4.03,34.70}, + {0.03237,0.00,2.180,0,0.4580,6.9980,45.80,6.0622,3,222.0,18.70,394.63,2.94,33.40}, + {0.06905,0.00,2.180,0,0.4580,7.1470,54.20,6.0622,3,222.0,18.70,396.90,5.33,36.20}, + {0.02985,0.00,2.180,0,0.4580,6.4300,58.70,6.0622,3,222.0,18.70,394.12,5.21,28.70}, + {0.08829,12.50,7.870,0,0.5240,6.0120,66.60,5.5605,5,311.0,15.20,395.60,12.43,22.90}, + {0.14455,12.50,7.870,0,0.5240,6.1720,96.10,5.9505,5,311.0,15.20,396.90,19.15,27.10}, + {0.21124,12.50,7.870,0,0.5240,5.6310,100.00,6.0821,5,311.0,15.20,386.63,29.93,16.50}, + {0.17004,12.50,7.870,0,0.5240,6.0040,85.90,6.5921,5,311.0,15.20,386.71,17.10,18.90}, + {0.22489,12.50,7.870,0,0.5240,6.3770,94.30,6.3467,5,311.0,15.20,392.52,20.45,15.00}, + {0.11747,12.50,7.870,0,0.5240,6.0090,82.90,6.2267,5,311.0,15.20,396.90,13.27,18.90}, + {0.09378,12.50,7.870,0,0.5240,5.8890,39.00,5.4509,5,311.0,15.20,390.50,15.71,21.70}, + {0.62976,0.00,8.140,0,0.5380,5.9490,61.80,4.7075,4,307.0,21.00,396.90,8.26,20.40}, + {0.63796,0.00,8.140,0,0.5380,6.0960,84.50,4.4619,4,307.0,21.00,380.02,10.26,18.20}, + {0.62739,0.00,8.140,0,0.5380,5.8340,56.50,4.4986,4,307.0,21.00,395.62,8.47,19.90}, + {1.05393,0.00,8.140,0,0.5380,5.9350,29.30,4.4986,4,307.0,21.00,386.85,6.58,23.10}, + {0.78420,0.00,8.140,0,0.5380,5.9900,81.70,4.2579,4,307.0,21.00,386.75,14.67,17.50}, + {0.80271,0.00,8.140,0,0.5380,5.4560,36.60,3.7965,4,307.0,21.00,288.99,11.69,20.20}, + {0.72580,0.00,8.140,0,0.5380,5.7270,69.50,3.7965,4,307.0,21.00,390.95,11.28,18.20}, + {1.25179,0.00,8.140,0,0.5380,5.5700,98.10,3.7979,4,307.0,21.00,376.57,21.02,13.60}, + {0.85204,0.00,8.140,0,0.5380,5.9650,89.20,4.0123,4,307.0,21.00,392.53,13.83,19.60}, + {1.23247,0.00,8.140,0,0.5380,6.1420,91.70,3.9769,4,307.0,21.00,396.90,18.72,15.20}, + {0.98843,0.00,8.140,0,0.5380,5.8130,100.00,4.0952,4,307.0,21.00,394.54,19.88,14.50} + }; + +} + From 63d5a292e31f991567697229211244b8f2196d00 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Fri, 31 Aug 2018 12:21:43 +0300 Subject: [PATCH 25/95] IGNITE-9424 Set proper partition for key during insert - Fixes #4645. --- .../cache/IgniteCacheOffheapManagerImpl.java | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 2fe097c3c5865..91c6f3e222bae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -101,6 +101,7 @@ import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX; @@ -1646,7 +1647,7 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo @Nullable CacheDataRow oldRow) throws IgniteCheckedException { int cacheId = grp.storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - DataRow dataRow = new DataRow(key, val, ver, partId, expireTime, cacheId); + DataRow dataRow = makeDataRow(key, val, ver, expireTime, cacheId); if (canUpdateOldRow(cctx, oldRow, dataRow) && rowStore.updateRow(oldRow.link(), dataRow)) dataRow.link(oldRow.link()); @@ -1667,6 +1668,22 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo return dataRow; } + /** + * @param key Cache key. + * @param val Cache value. + * @param ver Version. + * @param expireTime Expired time. + * @param cacheId Cache id. + * @return Made data row. + */ + @NotNull private DataRow makeDataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, long expireTime, + int cacheId) { + if (key.partition() == -1) + key.partition(partId); + + return new DataRow(key, val, ver, partId, expireTime, cacheId); + } + /** {@inheritDoc} */ @Override public boolean mvccInitialValue( GridCacheContext cctx, @@ -2342,10 +2359,7 @@ else if (res == ResultType.PREV_NOT_NULL) { assert oldRow == null || oldRow.cacheId() == cacheId : oldRow; - if (key.partition() == -1) - key.partition(partId); - - DataRow dataRow = new DataRow(key, val, ver, partId, expireTime, cacheId); + DataRow dataRow = makeDataRow(key, val, ver, expireTime, cacheId); CacheObjectContext coCtx = cctx.cacheObjectContext(); From 77b06a692e0a2755f070a16b453d3073bb1102d9 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Fri, 31 Aug 2018 13:28:40 +0300 Subject: [PATCH 26/95] IGNITE-9444 Mute failing test --- .../tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java b/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java index 4bddb18941762..c754553ef66e4 100644 --- a/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java +++ b/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java @@ -81,6 +81,8 @@ public void testGoogleComputeEngine() throws Exception { * @throws Exception If any error occurs. */ public void testRackspace() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-9444"); + testCloudProvider("rackspace-cloudservers-us"); } From e76e3a344e4902de41e390f2f35ff813b090a237 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Fri, 31 Aug 2018 17:49:55 +0700 Subject: [PATCH 27/95] IGNITE-8568 Web Console: Added support for "Collocated" query mode on "Queries" screen. --- .../page-queries/components/queries-notebook/controller.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js index 5ba7a2706bf07..d6a8271f2e6dc 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js @@ -41,7 +41,7 @@ const ROW_IDX = {value: -2, type: 'java.lang.Integer', label: 'ROW_IDX'}; const NON_COLLOCATED_JOINS_SINCE = '1.7.0'; -const COLLOCATED_QUERY_SINCE = [['2.3.5', '2.4.0'], ['2.4.6', '2.5.0'], '2.5.2']; +const COLLOCATED_QUERY_SINCE = [['2.3.5', '2.4.0'], ['2.4.6', '2.5.0'], '2.5.1-p13']; const ENFORCE_JOIN_SINCE = [['1.7.9', '1.8.0'], ['1.8.4', '1.9.0'], '1.9.1']; From 067df1345eae60cb6732709c8ef252b596a2a3f2 Mon Sep 17 00:00:00 2001 From: NSAmelchev Date: Fri, 31 Aug 2018 15:47:16 +0300 Subject: [PATCH 28/95] IGNITE-8189 Improved ZkDistributedCollectDataFuture#deleteFutureData implementation - Fixes #4537. Signed-off-by: Alexey Goncharuk --- .../ZkDistributedCollectDataFuture.java | 19 ++++----- .../zk/internal/ZookeeperClient.java | 40 +++++++------------ .../zk/internal/ZookeeperDiscoveryImpl.java | 39 ++++++------------ .../zk/internal/ZookeeperClientTest.java | 13 ++---- 4 files changed, 40 insertions(+), 71 deletions(-) diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java index e9b28e192216c..e710055d05757 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java @@ -18,6 +18,7 @@ package org.apache.ignite.spi.discovery.zk.internal; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Set; import java.util.UUID; @@ -145,23 +146,19 @@ static void deleteFutureData(ZookeeperClient client, UUID futId, IgniteLogger log ) throws Exception { - // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189 + List batch = new LinkedList<>(); + String evtDir = paths.distributedFutureBasePath(futId); - try { - client.deleteAll(evtDir, - client.getChildrenIfPathExists(evtDir), - -1); - } - catch (KeeperException.NoNodeException e) { - U.log(log, "Node for deletion was not found: " + e.getPath()); + if (client.exists(evtDir)) { + batch.addAll(client.getChildrenPaths(evtDir)); - // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189 + batch.add(evtDir); } - client.deleteIfExists(evtDir, -1); + batch.add(paths.distributedFutureResultPath(futId)); - client.deleteIfExists(paths.distributedFutureResultPath(futId), -1); + client.deleteAll(batch, -1); } /** diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java index b58f0cea0db5b..39417c2db96d2 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java @@ -504,9 +504,7 @@ String createSequential(String checkPrefix, String parentPath, String path, byte * @throws ZookeeperClientFailedException If connection to zk was lost. * @throws InterruptedException If interrupted. */ - List getChildren(String path) - throws ZookeeperClientFailedException, InterruptedException - { + List getChildren(String path) throws ZookeeperClientFailedException, InterruptedException { for (;;) { long connStartTime = this.connStartTime; @@ -520,29 +518,23 @@ List getChildren(String path) } /** + * Get children paths. + * * @param path Path. - * @return Children nodes. - * @throws KeeperException.NoNodeException If provided path does not exist. + * @return Children paths. * @throws ZookeeperClientFailedException If connection to zk was lost. * @throws InterruptedException If interrupted. */ - List getChildrenIfPathExists(String path) throws - KeeperException.NoNodeException, InterruptedException, ZookeeperClientFailedException { - for (;;) { - long connStartTime = this.connStartTime; + List getChildrenPaths(String path) throws ZookeeperClientFailedException, InterruptedException { + List children = getChildren(path); - try { - return zk.getChildren(path, false); - } - catch (KeeperException.NoNodeException e) { - throw e; - } - catch (Exception e) { - onZookeeperError(connStartTime, e); - } - } - } + ArrayList paths = new ArrayList(children.size()); + for (String child : children) + paths.add(path + "/" + child); + + return paths; + } /** * @param path Path. @@ -593,7 +585,7 @@ void deleteIfExists(String path, int ver) * @throws ZookeeperClientFailedException If connection to zk was lost. * @throws InterruptedException If interrupted. */ - void deleteAll(@Nullable String parent, List paths, int ver) + void deleteAll(List paths, int ver) throws ZookeeperClientFailedException, InterruptedException { if (paths.isEmpty()) return; @@ -605,10 +597,8 @@ void deleteAll(@Nullable String parent, List paths, int ver) List batch = new LinkedList<>(); for (String path : paths) { - String path0 = parent != null ? parent + "/" + path : path; - //TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8187 - int size = requestOverhead(path0) + 17 /* overhead */; + int size = requestOverhead(path) + 17 /* overhead */; assert size <= MAX_REQ_SIZE; @@ -620,7 +610,7 @@ void deleteAll(@Nullable String parent, List paths, int ver) batchSize = 0; } - batch.add(Op.delete(path0, ver)); + batch.add(Op.delete(path, ver)); batchSize += size; } diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java index 4579efd374906..069b3e24951e7 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -2263,33 +2264,27 @@ private void cleanupPreviousClusterData(long startInternalOrder) throws Exceptio ZookeeperClient client = rtState.zkClient; - // TODO ZK: use multi, better batching + max-size safe + NoNodeException safe. - List evtChildren = rtState.zkClient.getChildren(zkPaths.evtsPath); + List batch = new LinkedList<>(); - for (String evtPath : evtChildren) { - String evtDir = zkPaths.evtsPath + "/" + evtPath; + List evtChildren = client.getChildrenPaths(zkPaths.evtsPath); - removeChildren(evtDir); - } + for (String evtPath : evtChildren) + batch.addAll(client.getChildrenPaths(evtPath)); + + batch.addAll(evtChildren); - client.deleteAll(zkPaths.evtsPath, evtChildren, -1); + batch.addAll(client.getChildrenPaths(zkPaths.customEvtsDir)); - client.deleteAll(zkPaths.customEvtsDir, - client.getChildren(zkPaths.customEvtsDir), - -1); + batch.addAll(client.getChildrenPaths(zkPaths.customEvtsPartsDir)); - rtState.zkClient.deleteAll(zkPaths.customEvtsPartsDir, - rtState.zkClient.getChildren(zkPaths.customEvtsPartsDir), - -1); + batch.addAll(client.getChildrenPaths(zkPaths.customEvtsAcksDir)); - rtState.zkClient.deleteAll(zkPaths.customEvtsAcksDir, - rtState.zkClient.getChildren(zkPaths.customEvtsAcksDir), - -1); + client.deleteAll(batch, -1); if (startInternalOrder > 0) { - for (String alive : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) { + for (String alive : client.getChildren(zkPaths.aliveNodesDir)) { if (ZkIgnitePaths.aliveInternalId(alive) < startInternalOrder) - rtState.zkClient.deleteIfExists(zkPaths.aliveNodesDir + "/" + alive, -1); + client.deleteIfExists(zkPaths.aliveNodesDir + "/" + alive, -1); } } @@ -2301,14 +2296,6 @@ private void cleanupPreviousClusterData(long startInternalOrder) throws Exceptio } } - /** - * @param path Path. - * @throws Exception If failed. - */ - private void removeChildren(String path) throws Exception { - rtState.zkClient.deleteAll(path, rtState.zkClient.getChildren(path), -1); - } - /** * @param zkClient Client. * @param evtPath Event path. diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java index 0d649801a60be..7c9ec51b094b6 100644 --- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java @@ -196,12 +196,12 @@ public void testDeleteAll() throws Exception { client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT); client.createIfNeeded("/apacheIgnite/2", null, CreateMode.PERSISTENT); - client.deleteAll("/apacheIgnite", Arrays.asList("1", "2"), -1); + client.deleteAll(Arrays.asList("/apacheIgnite/1", "/apacheIgnite/2"), -1); assertTrue(client.getChildren("/apacheIgnite").isEmpty()); client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT); - client.deleteAll("/apacheIgnite", Collections.singletonList("1"), -1); + client.deleteAll(Collections.singletonList("/apacheIgnite/1"), -1); assertTrue(client.getChildren("/apacheIgnite").isEmpty()); } @@ -227,12 +227,7 @@ public void testDeleteAllRequestOverflow() throws Exception { assertEquals(cnt, client.getChildren("/apacheIgnite").size()); - List subPaths = new ArrayList<>(cnt); - - for (int i = 0; i < cnt; i++) - subPaths.add(String.valueOf(i)); - - client.deleteAll("/apacheIgnite", subPaths, -1); + client.deleteAll(paths, -1); assertTrue(client.getChildren("/apacheIgnite").isEmpty()); } @@ -249,7 +244,7 @@ public void testDeleteAllNoNode() throws Exception { client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT); client.createIfNeeded("/apacheIgnite/2", null, CreateMode.PERSISTENT); - client.deleteAll("/apacheIgnite", Arrays.asList("1", "2", "3"), -1); + client.deleteAll(Arrays.asList("/apacheIgnite/1", "/apacheIgnite/2", "/apacheIgnite/3"), -1); assertTrue(client.getChildren("/apacheIgnite").isEmpty()); } From aa9e922901b202bbbe15135e6d6fc35e3bca2e3f Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Fri, 31 Aug 2018 17:55:17 +0300 Subject: [PATCH 29/95] IGNITE-9273 Log logical records to WAL for LOCAL caches --- .../processors/cache/GridCacheMapEntry.java | 4 + .../cache/IgniteCacheOffheapManagerImpl.java | 2 +- .../GridCacheDatabaseSharedManager.java | 2 +- .../db/wal/IgniteWalRecoveryTest.java | 241 +++++++++--------- .../IgniteWalRecoveryWithCompactionTest.java | 0 5 files changed, 129 insertions(+), 120 deletions(-) rename modules/{core => indexing}/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java (89%) rename modules/{core => indexing}/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryWithCompactionTest.java (100%) 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 249fbb3a14ec7..4dc49441c2347 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 @@ -2037,6 +2037,8 @@ else if (ttl != CU.TTL_ZERO) update(updated, expireTime, ttl, ver, true); + logUpdate(op, updated, ver, expireTime, 0); + if (evt) { CacheObject evtOld = null; @@ -2067,6 +2069,8 @@ else if (ttl != CU.TTL_ZERO) update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, ver, true); + logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0); + if (evt) { CacheObject evtOld = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 91c6f3e222bae..bc35264e79097 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -295,7 +295,7 @@ private void removeCacheData(int cacheId) { * @param part Partition. * @return Data store for given entry. */ - public CacheDataStore dataStore(GridDhtLocalPartition part) { + @Override public CacheDataStore dataStore(GridDhtLocalPartition part) { if (grp.isLocal()) return locCacheDataStore; else { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 7393c017a5149..2e5f26fb4c4bb 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -2492,7 +2492,7 @@ private void applyUpdate(GridCacheContext cacheCtx, DataEntry dataEntry) throws if (partId == -1) partId = cacheCtx.affinity().partition(dataEntry.key()); - GridDhtLocalPartition locPart = cacheCtx.topology().forceCreatePartition(partId); + GridDhtLocalPartition locPart = cacheCtx.isLocal() ? null : cacheCtx.topology().forceCreatePartition(partId); switch (dataEntry.op()) { case CREATE: diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java similarity index 89% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java index 116f174e83044..0b3ded493b6b9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java @@ -109,7 +109,16 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { private boolean fork; /** */ - private String cacheName; + private static final String CACHE_NAME = "partitioned"; + + /** */ + private static final String RENAMED_CACHE_NAME = "partitioned0"; + + /** */ + private static final String LOC_CACHE_NAME = "local"; + + /** */ + private boolean renamed; /** */ private int walSegmentSize; @@ -126,7 +135,8 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - CacheConfiguration ccfg = new CacheConfiguration<>(cacheName); + CacheConfiguration ccfg = renamed ? + new CacheConfiguration<>(RENAMED_CACHE_NAME) : new CacheConfiguration<>(CACHE_NAME); ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); ccfg.setRebalanceMode(CacheRebalanceMode.SYNC); @@ -134,7 +144,11 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { ccfg.setNodeFilter(new RemoteNodeFilter()); ccfg.setIndexedTypes(Integer.class, IndexedObject.class); - cfg.setCacheConfiguration(ccfg); + CacheConfiguration locCcfg = new CacheConfiguration<>(LOC_CACHE_NAME); + locCcfg.setCacheMode(CacheMode.LOCAL); + locCcfg.setIndexedTypes(Integer.class, IndexedObject.class); + + cfg.setCacheConfiguration(ccfg, locCcfg); DataStorageConfiguration dbCfg = new DataStorageConfiguration(); @@ -161,8 +175,6 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { cfg.setDataStorageConfiguration(dbCfg); - cfg.setMarshaller(null); - BinaryConfiguration binCfg = new BinaryConfiguration(); binCfg.setCompactFooter(false); @@ -181,7 +193,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { cleanPersistenceDir(); - cacheName = "partitioned"; + renamed = false; } /** {@inheritDoc} */ @@ -199,9 +211,9 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest { public void testWalBig() throws Exception { IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); Random rnd = new Random(); @@ -226,9 +238,9 @@ public void testWalBig() throws Exception { ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - cache = ignite.cache("partitioned"); + cache = ignite.cache(CACHE_NAME); // Check. for (Integer k : map.keySet()) @@ -243,9 +255,9 @@ public void testWalBigObjectNodeCancel() throws Exception { IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); for (int i = 0; i < MAX_SIZE_POWER; ++i) { int size = 1 << i; @@ -257,9 +269,9 @@ public void testWalBigObjectNodeCancel() throws Exception { ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - cache = ignite.cache("partitioned"); + cache = ignite.cache(CACHE_NAME); // Check. for (int i = 0; i < MAX_SIZE_POWER; ++i) { @@ -283,12 +295,9 @@ public void testSwitchClassLoader() throws Exception { // CustomDiscoveryMessage will trigger service tasks startGrid(2); - igniteEx.active(true); - - IgniteCache cache = igniteEx.cache("partitioned"); + igniteEx.cluster().active(true); - // Creates LoadCacheJobV2 -// cache.loadCache(null); + IgniteCache cache = igniteEx.cache(CACHE_NAME); final ClassLoader oldCl = Thread.currentThread().getContextClassLoader(); final ClassLoader newCl = getExternalClassLoader(); @@ -324,17 +333,14 @@ public void testWalSimple() throws Exception { try { IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); info(" --> step1"); - for (int i = 0; i < 10_000; i += 2) { -// X.println(" -> put: " + i); - + for (int i = 0; i < 10_000; i += 2) cache.put(i, new IndexedObject(i)); - } info(" --> step2"); @@ -368,9 +374,9 @@ else if (i % 2 == 0) ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - cache = ignite.cache("partitioned"); + cache = ignite.cache(CACHE_NAME); info(" --> check2"); @@ -404,9 +410,9 @@ public void testWalLargeValue() throws Exception { try { IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); for (int i = 0; i < 10_000; i++) { final byte[] data = new byte[i]; @@ -417,29 +423,15 @@ public void testWalLargeValue() throws Exception { if (i % 1000 == 0) X.println(" ---> put: " + i); - -// Assert.assertArrayEquals(data, (byte[])cache.get(i)); } -// info(" --> check1"); -// -// for (int i = 0; i < 25_000; i++) { -// final byte[] data = new byte[i]; -// -// Arrays.fill(data, (byte)i); -// -// final byte[] loaded = (byte[]) cache.get(i); -// -// Assert.assertArrayEquals(data, loaded); -// } - stopGrid(1); ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - cache = ignite.cache("partitioned"); + cache = ignite.cache(CACHE_NAME); info(" --> check2"); @@ -486,7 +478,7 @@ public void testHugeCheckpointRecord() throws Exception { try { final IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); for (int i = 0; i < 50; i++) { CacheConfiguration ccfg = new CacheConfiguration<>("cache-" + i); @@ -502,7 +494,7 @@ public void testHugeCheckpointRecord() throws Exception { final long endTime = System.currentTimeMillis() + 30_000; IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { - @Override public Void call() throws Exception { + @Override public Void call() { Random rnd = ThreadLocalRandom.current(); while (U.currentTimeMillis() < endTime) { @@ -534,17 +526,17 @@ public void testHugeCheckpointRecord() throws Exception { private void checkWalRolloverMultithreaded() throws Exception { walSegmentSize = 2 * 1024 * 1024; - final long endTime = System.currentTimeMillis() + 2 * 60 * 1000; + final long endTime = System.currentTimeMillis() + 60 * 1000; try { IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - final IgniteCache cache = ignite.cache("partitioned"); + final IgniteCache cache = ignite.cache(CACHE_NAME); GridTestUtils.runMultiThreaded(new Callable() { - @Override public Void call() throws Exception { + @Override public Void call() { Random rnd = ThreadLocalRandom.current(); while (U.currentTimeMillis() < endTime) @@ -566,9 +558,9 @@ public void testWalRenameDirSimple() throws Exception { try { IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); for (int i = 0; i < 100; i++) cache.put(i, new IndexedObject(i)); @@ -577,19 +569,17 @@ public void testWalRenameDirSimple() throws Exception { stopGrid(1); - final File cacheDir = cacheDir("partitioned", consistentId.toString()); + final File cacheDir = cacheDir(CACHE_NAME, consistentId.toString()); - final boolean renamed = cacheDir.renameTo(new File(cacheDir.getParent(), "cache-partitioned0")); + renamed = cacheDir.renameTo(new File(cacheDir.getParent(), "cache-" + RENAMED_CACHE_NAME)); assert renamed; - cacheName = "partitioned0"; - ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); - cache = ignite.cache(cacheName); + cache = ignite.cache(RENAMED_CACHE_NAME); for (int i = 0; i < 100; i++) assertEquals(new IndexedObject(i), cache.get(i)); @@ -635,7 +625,7 @@ public void testRecoveryNoCheckpoint() throws Exception { IgniteEx cacheGrid = startGrid(1); - ctrlGrid.active(true); + ctrlGrid.cluster().active(true); ctrlGrid.compute(ctrlGrid.cluster().forRemotes()).run(new LoadRunnable(false)); @@ -657,7 +647,7 @@ public void testRecoveryNoCheckpoint() throws Exception { // Now start the grid and verify that updates were restored from WAL. cacheGrid = startGrid(1); - IgniteCache cache = cacheGrid.cache("partitioned"); + IgniteCache cache = cacheGrid.cache(CACHE_NAME); for (int i = 0; i < 10_000; i++) assertEquals(new IndexedObject(i), cache.get(i)); @@ -666,6 +656,11 @@ public void testRecoveryNoCheckpoint() throws Exception { assertEquals(1, res.size()); assertEquals(10_000L, res.get(0).get(0)); + + IgniteCache locCache = cacheGrid.cache(LOC_CACHE_NAME); + + for (int i = 0; i < 10_000; i++) + assertEquals(new IndexedObject(i), locCache.get(i)); } finally { stopAllGrids(); @@ -683,7 +678,7 @@ public void testRecoveryLargeNoCheckpoint() throws Exception { IgniteEx cacheGrid = startGrid(1); - ctrlGrid.active(true); + ctrlGrid.cluster().active(true); ctrlGrid.compute(ctrlGrid.cluster().forRemotes()).run(new LargeLoadRunnable(false)); @@ -705,16 +700,16 @@ public void testRecoveryLargeNoCheckpoint() throws Exception { // Now start the grid and verify that updates were restored from WAL. cacheGrid = startGrid(1); - IgniteCache cache = cacheGrid.cache("partitioned"); + IgniteCache cache = cacheGrid.cache(CACHE_NAME); + IgniteCache locCache = cacheGrid.cache(LOC_CACHE_NAME); for (int i = 0; i < 1000; i++) { final long[] data = new long[LARGE_ARR_SIZE]; Arrays.fill(data, i); - final long[] loaded = (long[]) cache.get(i); - - Assert.assertArrayEquals(data, loaded); + Assert.assertArrayEquals(data, (long[])cache.get(i)); + Assert.assertArrayEquals(data, (long[])locCache.get(i)); } } finally { @@ -738,7 +733,7 @@ public void testRandomCrash() throws Exception { IgniteEx cacheGrid = startGrid(1); - ctrlGrid.active(true); + ctrlGrid.cluster().active(true); IgniteCompute rmt = ctrlGrid.compute(ctrlGrid.cluster().forRemotes()); @@ -776,7 +771,7 @@ public void testLargeRandomCrash() throws Exception { IgniteEx cacheGrid = startGrid(1); - ctrlGrid.active(true); + ctrlGrid.cluster().active(true); IgniteCompute rmt = ctrlGrid.compute(ctrlGrid.cluster().forRemotes()); @@ -820,7 +815,7 @@ public void testDestroyCache() throws Exception { try { IgniteEx ignite = startGrid(1); - ignite.active(true); + ignite.cluster().active(true); IgniteCache cache = ignite.getOrCreateCache("test"); @@ -845,16 +840,16 @@ public void testEvictPartition() throws Exception { try { Ignite ignite1 = startGrid("node1"); - ignite1.active(true); + ignite1.cluster().active(true); - IgniteCache cache1 = ignite1.cache(cacheName); + IgniteCache cache1 = ignite1.cache(CACHE_NAME); for (int i = 0; i < 100; i++) cache1.put(i, new IndexedObject(i)); Ignite ignite2 = startGrid("node2"); - IgniteCache cache2 = ignite2.cache(cacheName); + IgniteCache cache2 = ignite2.cache(CACHE_NAME); for (int i = 0; i < 100; i++) { assertEquals(new IndexedObject(i), cache1.get(i)); @@ -867,10 +862,10 @@ public void testEvictPartition() throws Exception { ignite1 = startGrid("node1"); ignite2 = startGrid("node2"); - ignite1.active(true); + ignite1.cluster().active(true); - cache1 = ignite1.cache(cacheName); - cache2 = ignite2.cache(cacheName); + cache1 = ignite1.cache(CACHE_NAME); + cache2 = ignite2.cache(CACHE_NAME); for (int i = 0; i < 100; i++) { assertEquals(new IndexedObject(i), cache1.get(i)); @@ -892,7 +887,7 @@ public void testMetastorage() throws Exception { IgniteEx ignite0 = (IgniteEx)startGrid("node1"); IgniteEx ignite1 = (IgniteEx)startGrid("node2"); - ignite1.active(true); + ignite1.cluster().active(true); GridCacheSharedContext sharedCtx0 = ignite0.context().cache().context(); GridCacheSharedContext sharedCtx1 = ignite1.context().cache().context(); @@ -957,7 +952,7 @@ public void testMetastorageLargeArray() throws Exception { IgniteEx ignite = (IgniteEx)startGrid("node1"); - ignite.active(true); + ignite.cluster().active(true); GridCacheSharedContext sharedCtx = ignite.context().cache().context(); @@ -1003,7 +998,7 @@ public void testMetastorageRemove() throws Exception { IgniteEx ignite0 = (IgniteEx)startGrid("node1"); - ignite0.active(true); + ignite0.cluster().active(true); GridCacheSharedContext sharedCtx0 = ignite0.context().cache().context(); @@ -1056,7 +1051,7 @@ public void testMetastorageUpdate() throws Exception { IgniteEx ignite0 = (IgniteEx)startGrid("node1"); - ignite0.active(true); + ignite0.cluster().active(true); GridCacheSharedContext sharedCtx0 = ignite0.context().cache().context(); @@ -1106,9 +1101,9 @@ public void testMetastorageWalRestore() throws Exception { try { int cnt = 2000; - IgniteEx ignite0 = (IgniteEx)startGrid(0); + IgniteEx ignite0 = startGrid(0); - ignite0.active(true); + ignite0.cluster().active(true); GridCacheSharedContext sharedCtx0 = ignite0.context().cache().context(); @@ -1137,7 +1132,7 @@ public void testMetastorageWalRestore() throws Exception { ignite0 = startGrid(0); - ignite0.active(true); + ignite0.cluster().active(true); sharedCtx0 = ignite0.context().cache().context(); @@ -1162,9 +1157,9 @@ public void testApplyDeltaRecords() throws Exception { try { IgniteEx ignite0 = (IgniteEx)startGrid("node0"); - ignite0.active(true); + ignite0.cluster().active(true); - IgniteCache cache0 = ignite0.cache(cacheName); + IgniteCache cache0 = ignite0.cache(CACHE_NAME); for (int i = 0; i < 1000; i++) cache0.put(i, new IndexedObject(i)); @@ -1297,7 +1292,7 @@ else if (rec instanceof PageDeltaRecord) { */ public void testRecoveryOnTransactionalAndPartitionedCache() throws Exception { IgniteEx ignite = (IgniteEx) startGrids(3); - ignite.active(true); + ignite.cluster().active(true); try { final String cacheName = "transactional"; @@ -1329,11 +1324,7 @@ public void testRecoveryOnTransactionalAndPartitionedCache() throws Exception { for (int op = 0; op < operationsPerTransaction; op++) { int key = random.nextInt(1000) + 1; - Object value; - if (random.nextBoolean()) - value = randomString(random) + key; - else - value = new BigObject(key); + Object value = random.nextBoolean() ? randomString(random) + key : new BigObject(key); changesInTransaction.put(key, value); @@ -1355,7 +1346,7 @@ public void testRecoveryOnTransactionalAndPartitionedCache() throws Exception { stopAllGrids(); ignite = (IgniteEx) startGrids(3); - ignite.active(true); + ignite.cluster().active(true); cache = ignite.cache(cacheName); @@ -1379,7 +1370,7 @@ public void testTxRecordsConsistency() throws Exception { System.setProperty(IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS, "true"); IgniteEx ignite = (IgniteEx) startGrids(3); - ignite.active(true); + ignite.cluster().active(true); try { final String cacheName = "transactional"; @@ -1418,11 +1409,7 @@ public void testTxRecordsConsistency() throws Exception { for (int op = 0; op < operationsPerTransaction; op++) { int key = random.nextInt(1000) + 1; - Object value; - if (random.nextBoolean()) - value = randomString(random) + key; - else - value = new BigObject(key); + Object value = random.nextBoolean() ? randomString(random) + key : new BigObject(key); cache.put(key, value); } @@ -1583,7 +1570,7 @@ private LoadRunnable(boolean disableCheckpoints) { try { boolean successfulWaiting = GridTestUtils.waitForCondition(new PAX() { @Override public boolean applyx() { - return ignite.cache("partitioned") != null; + return ignite.cache(CACHE_NAME) != null; } }, 10_000); @@ -1593,10 +1580,13 @@ private LoadRunnable(boolean disableCheckpoints) { throw new RuntimeException(e); } - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); + IgniteCache locCache = ignite.cache(LOC_CACHE_NAME); - for (int i = 0; i < 10_000; i++) + for (int i = 0; i < 10_000; i++) { cache.put(i, new IndexedObject(i)); + locCache.put(i, new IndexedObject(i)); + } ignite.log().info("Finished load."); } @@ -1615,7 +1605,7 @@ private static class AsyncLoadRunnable implements IgniteRunnable { try { boolean successfulWaiting = GridTestUtils.waitForCondition(new PAX() { @Override public boolean applyx() { - return ignite.cache("partitioned") != null; + return ignite.cache(CACHE_NAME) != null; } }, 10_000); @@ -1629,8 +1619,9 @@ private static class AsyncLoadRunnable implements IgniteRunnable { for (int i = 0; i < 4; i++) { ignite.scheduler().callLocal(new Callable() { - @Override public Object call() throws Exception { - IgniteCache cache = ignite.cache("partitioned"); + @Override public Object call() { + IgniteCache cache = ignite.cache(CACHE_NAME); + IgniteCache locCache = ignite.cache(LOC_CACHE_NAME); ThreadLocalRandom rnd = ThreadLocalRandom.current(); @@ -1638,6 +1629,7 @@ private static class AsyncLoadRunnable implements IgniteRunnable { while (!Thread.currentThread().isInterrupted()) { cache.put(rnd.nextInt(10_000), new IndexedObject(rnd.nextInt())); + locCache.put(rnd.nextInt(10_000), new IndexedObject(rnd.nextInt())); cnt++; @@ -1665,7 +1657,7 @@ private static class VerifyCallable implements IgniteCallable { try { boolean successfulWaiting = GridTestUtils.waitForCondition(new PAX() { @Override public boolean applyx() { - return ignite.cache("partitioned") != null; + return ignite.cache(CACHE_NAME) != null; } }, 10_000); @@ -1675,15 +1667,28 @@ private static class VerifyCallable implements IgniteCallable { throw new RuntimeException(e); } - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); + IgniteCache locCache = ignite.cache(LOC_CACHE_NAME); for (int i = 0; i < 10_000; i++) { - Object val = cache.get(i); + { + Object val = cache.get(i); - if (val == null) { - ignite.log().warning("Failed to find a value for key: " + i); + if (val == null) { + ignite.log().warning("Failed to find a value for PARTITIONED cache key: " + i); - return false; + return false; + } + } + + { + Object val = locCache.get(i); + + if (val == null) { + ignite.log().warning("Failed to find a value for LOCAL cache key: " + i); + + return false; + } } } @@ -1714,7 +1719,7 @@ private LargeLoadRunnable(boolean disableCheckpoints) { try { boolean successfulWaiting = GridTestUtils.waitForCondition(new PAX() { @Override public boolean applyx() { - return ignite.cache("partitioned") != null; + return ignite.cache(CACHE_NAME) != null; } }, 10_000); @@ -1733,7 +1738,8 @@ private LargeLoadRunnable(boolean disableCheckpoints) { dbMgr.enableCheckpoints(false); } - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); + IgniteCache locCache = ignite.cache(LOC_CACHE_NAME); for (int i = 0; i < 1000; i++) { final long[] data = new long[LARGE_ARR_SIZE]; @@ -1741,6 +1747,7 @@ private LargeLoadRunnable(boolean disableCheckpoints) { Arrays.fill(data, i); cache.put(i, data); + locCache.put(i, data); } ignite.log().info("Finished load."); @@ -1760,7 +1767,7 @@ private static class AsyncLargeLoadRunnable implements IgniteRunnable { try { boolean successfulWaiting = GridTestUtils.waitForCondition(new PAX() { @Override public boolean applyx() { - return ignite.cache("partitioned") != null; + return ignite.cache(CACHE_NAME) != null; } }, 10_000); @@ -1774,8 +1781,8 @@ private static class AsyncLargeLoadRunnable implements IgniteRunnable { for (int i = 0; i < 1; i++) { ignite.scheduler().callLocal(new Callable() { - @Override public Object call() throws Exception { - IgniteCache cache = ignite.cache("partitioned"); + @Override public Object call() { + IgniteCache cache = ignite.cache(CACHE_NAME); ThreadLocalRandom rnd = ThreadLocalRandom.current(); @@ -1788,8 +1795,6 @@ private static class AsyncLargeLoadRunnable implements IgniteRunnable { Arrays.fill(data, key); -// System.out.println("> " + key); - cache.put(key, data); cnt++; @@ -1818,7 +1823,7 @@ private static class VerifyLargeCallable implements IgniteCallable { try { boolean successfulWaiting = GridTestUtils.waitForCondition(new PAX() { @Override public boolean applyx() { - return ignite.cache("partitioned") != null; + return ignite.cache(CACHE_NAME) != null; } }, 10_000); @@ -1828,7 +1833,7 @@ private static class VerifyLargeCallable implements IgniteCallable { throw new RuntimeException(e); } - IgniteCache cache = ignite.cache("partitioned"); + IgniteCache cache = ignite.cache(CACHE_NAME); for (int i = 0; i < 1000; i++) { final long[] data = new long[LARGE_ARR_SIZE]; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryWithCompactionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryWithCompactionTest.java similarity index 100% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryWithCompactionTest.java rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryWithCompactionTest.java From fa1a6d6c00de0b6f1233bdcdc8b96183b38bff41 Mon Sep 17 00:00:00 2001 From: vd-pyatkov Date: Fri, 31 Aug 2018 18:41:58 +0300 Subject: [PATCH 30/95] IGNITE-9448 Update ZooKeeper version to 3.4.13 - Fixes #4661. Signed-off-by: Alexey Goncharuk --- parent/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parent/pom.xml b/parent/pom.xml index 76e39b79e29fa..f5b49ba39a8a4 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -122,7 +122,7 @@ 2.2.0 0.8.3 0.5 - 3.4.6 + 3.4.13 * From cd8ce25b1947a66f6541b471831425f61f1ea5e1 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 31 Aug 2018 19:24:00 +0300 Subject: [PATCH 31/95] IGNITE-9373 Fixed MVCC failing tests - Fixes #4623. Signed-off-by: Alexey Goncharuk --- .../suite/IgniteJdbcDriverMvccTestSuite.java | 47 ++ .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 1 - ...inTransactionsWithMvccEnabledSelfTest.java | 447 ++++++++++++++++++ .../cache/IgniteCacheOffheapManagerImpl.java | 11 +- .../cache/mvcc/MvccProcessorImpl.java | 35 +- .../processors/cache/mvcc/MvccUtils.java | 11 +- .../cache/query/GridCacheQueryAdapter.java | 16 +- ...cAbstractBasicCoordinatorFailoverTest.java | 4 - .../mvcc/CacheMvccAbstractFeatureTest.java | 13 +- .../cache/mvcc/CacheMvccAbstractTest.java | 21 +- .../mvcc/CacheMvccClusterRestartTest.java | 18 +- .../cache/mvcc/CacheMvccTransactionsTest.java | 37 +- .../DataStreamProcessorMvccSelfTest.java | 70 +++ .../junits/common/GridCommonAbstractTest.java | 11 +- .../testsuites/IgniteCacheMvccTestSuite.java | 14 +- .../processors/query/h2/H2FieldsIterator.java | 2 +- .../query/h2/H2ResultSetIterator.java | 25 +- .../index/SqlTransactionsComandsSelfTest.java | 83 ++++ ...ctionsCommandsWithMvccEnabledSelfTest.java | 420 ++++++++++++++++ .../mvcc/CacheMvccBackupsAbstractTest.java | 6 +- .../mvcc/CacheMvccSqlQueriesAbstractTest.java | 2 + ...ccSqlTxQueriesWithReducerAbstractTest.java | 2 +- .../query/h2/GridIndexRebuildSelfTest.java | 188 ++------ ...idIndexRebuildWithMvccEnabledSelfTest.java | 126 +++++ .../IgniteCacheMvccSqlTestSuite.java | 22 +- .../IgniteCacheQuerySelfTestSuite.java | 14 +- .../IgniteConfigurationParityTest.cs | 5 +- 27 files changed, 1400 insertions(+), 251 deletions(-) create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsCommandsWithMvccEnabledSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java new file mode 100644 index 0000000000000..6d8933dfe143f --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.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.jdbc.suite; + +import junit.framework.TestSuite; +import org.apache.ignite.jdbc.thin.JdbcThinConnectionMvccEnabledSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientAutoCommitComplexSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientNoAutoCommitComplexSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsWithMvccEnabledSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerAutoCommitComplexSelfTest; +import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerNoAutoCommitComplexSelfTest; + +public class IgniteJdbcDriverMvccTestSuite extends TestSuite { + /** + * @return JDBC Driver Test Suite. + * @throws Exception In case of error. + */ + public static TestSuite suite() throws Exception { + TestSuite suite = new TestSuite("Ignite JDBC Driver Test Suite"); + + suite.addTest(new TestSuite(JdbcThinConnectionMvccEnabledSelfTest.class)); + + // Transactions + suite.addTest(new TestSuite(JdbcThinTransactionsWithMvccEnabledSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsClientAutoCommitComplexSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsServerAutoCommitComplexSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.class)); + + return suite; + } +} 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 275040f81619f..7fbf41fe2e55d 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 @@ -19,7 +19,6 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.jdbc2.JdbcBlobTest; -import org.apache.ignite.internal.jdbc2.JdbcBulkLoadSelfTest; import org.apache.ignite.internal.jdbc2.JdbcConnectionReopenTest; import org.apache.ignite.internal.jdbc2.JdbcDistributedJoinsQueryTest; import org.apache.ignite.jdbc.JdbcComplexQuerySelfTest; diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java new file mode 100644 index 0000000000000..e3f7f14571a90 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java @@ -0,0 +1,447 @@ +/* + * 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.jdbc.thin; + +import java.sql.BatchUpdateException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.processors.query.NestedTxMode; +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.GridStringLogger; +import org.apache.ignite.testframework.GridTestUtils; +import org.jetbrains.annotations.NotNull; + +/** + * Tests to check behavior with transactions on. + */ +public class JdbcThinTransactionsWithMvccEnabledSelfTest extends JdbcThinAbstractSelfTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String URL = "jdbc:ignite:thin://127.0.0.1"; + + /** Logger. */ + private GridStringLogger log; + + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setMvccEnabled(true); + + cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME)); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + + cfg.setMarshaller(new BinaryMarshaller()); + + cfg.setGridLogger(log = new GridStringLogger()); + + return cfg; + } + + /** + * @param name Cache name. + * @return Cache configuration. + * @throws Exception In case of error. + */ + private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception { + CacheConfiguration cfg = defaultCacheConfiguration(); + + cfg.setName(name); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + startGrid(0); + + try (Connection c = c(true, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + s.execute("CREATE TABLE INTS (k int primary key, v int) WITH \"cache_name=ints,wrap_value=false," + + "atomicity=transactional\""); + } + } + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @param autoCommit Auto commit mode. + * @param nestedTxMode Nested transactions mode. + * @return Connection. + * @throws SQLException if failed. + */ + private static Connection c(boolean autoCommit, NestedTxMode nestedTxMode) throws SQLException { + Connection res = DriverManager.getConnection(URL + "/?nestedTransactionsMode=" + nestedTxMode.name()); + + res.setAutoCommit(autoCommit); + + return res; + } + + /** + * + */ + public void testTransactionsBeginCommitRollback() throws IgniteCheckedException { + final AtomicBoolean stop = new AtomicBoolean(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + try { + try (Connection c = c(false, NestedTxMode.ERROR)) { + while (!stop.get()) { + try (Statement s = c.createStatement()) { + s.execute("BEGIN"); + + c.commit(); + + s.execute("BEGIN"); + + c.rollback(); + } + } + } + } + catch (SQLException e) { + throw new AssertionError(e); + } + } + }, 8, "jdbc-transactions"); + + U.sleep(5000); + + stop.set(true); + + fut.get(); + } + + /** + * + */ + public void testTransactionsBeginCommitRollbackAutocommit() throws IgniteCheckedException { + GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + try { + try (Connection c = c(true, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + s.execute("BEGIN"); + + s.execute("COMMIT"); + + s.execute("BEGIN"); + + s.execute("ROLLBACK"); + } + } + } + catch (SQLException e) { + throw new AssertionError(e); + } + } + }, 8, "jdbc-transactions").get(); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOff() throws SQLException { + try (Connection c = c(false, NestedTxMode.IGNORE)) { + doNestedTxStart(c, false); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOff() throws SQLException { + try (Connection c = c(false, NestedTxMode.COMMIT)) { + doNestedTxStart(c, false); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOff() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(false, NestedTxMode.ERROR)) { + doNestedTxStart(c, false); + } + + throw new AssertionError(); + } + }, SQLException.class, "Transaction has already been started."); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOn() throws SQLException { + try (Connection c = c(true, NestedTxMode.IGNORE)) { + doNestedTxStart(c, false); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOn() throws SQLException { + try (Connection c = c(true, NestedTxMode.COMMIT)) { + doNestedTxStart(c, false); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOn() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(true, NestedTxMode.ERROR)) { + doNestedTxStart(c, false); + } + + throw new AssertionError(); + } + }, SQLException.class, "Transaction has already been started."); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOffBatched() throws SQLException { + try (Connection c = c(false, NestedTxMode.IGNORE)) { + doNestedTxStart(c, true); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOffBatched() throws SQLException { + try (Connection c = c(false, NestedTxMode.COMMIT)) { + doNestedTxStart(c, true); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOffBatched() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(false, NestedTxMode.ERROR)) { + doNestedTxStart(c, true); + } + + throw new AssertionError(); + } + }, BatchUpdateException.class, "Transaction has already been started."); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIgnoreNestedTxAutocommitOnBatched() throws SQLException { + try (Connection c = c(true, NestedTxMode.IGNORE)) { + doNestedTxStart(c, true); + } + + assertTrue(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCommitNestedTxAutocommitOnBatched() throws SQLException { + try (Connection c = c(true, NestedTxMode.COMMIT)) { + doNestedTxStart(c, true); + } + + assertFalse(log.toString().contains("ignoring BEGIN command")); + } + + /** + * + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testErrorNestedTxAutocommitOnBatched() throws SQLException { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + try (Connection c = c(true, NestedTxMode.ERROR)) { + doNestedTxStart(c, true); + } + + throw new AssertionError(); + } + }, BatchUpdateException.class, "Transaction has already been started."); + } + + /** + * Try to start nested transaction via batch as well as separate statements. + * @param conn Connection. + * @param batched Whether {@link Statement#executeBatch()} should be used. + * @throws SQLException if failed. + */ + private void doNestedTxStart(Connection conn, boolean batched) throws SQLException { + try (Statement s = conn.createStatement()) { + s.executeQuery("SELECT * FROM INTS"); + + if (batched) { + s.addBatch("BEGIN"); + + s.addBatch("BEGIN"); + + s.executeBatch(); + } + else { + s.execute("BEGIN"); + + s.execute("BEGIN"); + } + } + } + + /** + * @throws SQLException if failed. + */ + public void testAutoCommitSingle() throws SQLException { + doTestAutoCommit(false); + } + + /** + * @throws SQLException if failed. + */ + public void testAutoCommitBatched() throws SQLException { + doTestAutoCommit(true); + } + + /** + * @param batched Batch mode flag. + * @throws SQLException if failed. + */ + private void doTestAutoCommit(boolean batched) throws SQLException { + IgniteCache cache = grid(0).cache("ints"); + + try (Connection c = c(false, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + assertFalse(s.executeQuery("SELECT * from INTS").next()); + + if (batched) { + s.addBatch("INSERT INTO INTS(k, v) values(1, 1)"); + + s.executeBatch(); + } + else + s.execute("INSERT INTO INTS(k, v) values(1, 1)"); + + // We haven't committed anything yet - this check shows that autoCommit flag is in effect. + assertTrue(cache.query(new SqlFieldsQuery("SELECT * from INTS")).getAll().isEmpty()); + + // We should see own updates. + assertTrue(s.executeQuery("SELECT * from INTS").next()); + + c.commit(); + + c.setAutoCommit(true); + + assertEquals(1, cache.get(1)); + + assertTrue(s.executeQuery("SELECT * from INTS").next()); + } + } + } + + /** + * Test that exception in one of the statements does not kill connection worker altogether. + * @throws SQLException if failed. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testExceptionHandling() throws SQLException { + try (Connection c = c(true, NestedTxMode.ERROR)) { + try (Statement s = c.createStatement()) { + s.execute("INSERT INTO INTS(k, v) values(1, 1)"); + + assertEquals(1, grid(0).cache("ints").get(1)); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + s.execute("INSERT INTO INTS(x, y) values(1, 1)"); + + return null; + } + }, SQLException.class, "Failed to parse query"); + + s.execute("INSERT INTO INTS(k, v) values(2, 2)"); + + assertEquals(2, grid(0).cache("ints").get(2)); + } + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index bc35264e79097..13ad7e2d5236c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -2565,7 +2565,9 @@ private void clearPendingEntries(GridCacheContext cctx, CacheDataRow oldRow) List> res = new ArrayList<>(); - long crd = MVCC_CRD_COUNTER_NA, cntr = MVCC_COUNTER_NA; int opCntr = MVCC_OP_COUNTER_NA; + long crd = MVCC_CRD_COUNTER_NA; + long cntr = MVCC_COUNTER_NA; + int opCntr = MVCC_OP_COUNTER_NA; while (cur.next()) { CacheDataRow row = cur.get(); @@ -2575,7 +2577,9 @@ private void clearPendingEntries(GridCacheContext cctx, CacheDataRow oldRow) res.add(F.t(row.value(), row.mvccVersion())); - crd = row.mvccCoordinatorVersion(); cntr = row.mvccCounter(); opCntr = row.mvccOperationCounter(); + crd = row.mvccCoordinatorVersion(); + cntr = row.mvccCounter(); + opCntr = row.mvccOperationCounter(); } return res; @@ -2654,8 +2658,7 @@ private void afterRowFound(@Nullable CacheDataRow row, KeyCacheObject key) throw return cursor(cacheId, null, null); } - /** {@inheritDoc} - * @param cacheId*/ + /** {@inheritDoc} */ @Override public GridCursor cursor(int cacheId, MvccSnapshot mvccSnapshot) throws IgniteCheckedException { return cursor(cacheId, null, null, null, mvccSnapshot); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java index 31d3b6115bc45..220f0c0bdfd19 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java @@ -545,7 +545,7 @@ public MvccProcessorImpl(GridKernalContext ctx) { } if (!ctx.localNodeId().equals(crd.nodeId()) || !initFut.isDone()) - return null; + return null; else if (tx != null) return assignTxSnapshot(0L); else @@ -1324,7 +1324,9 @@ private void processCoordinatorQuerySnapshotRequest(UUID nodeId, MvccQuerySnapsh * @param msg Message. */ private void processCoordinatorSnapshotResponse(UUID nodeId, MvccSnapshotResponse msg) { - Map map = snapLsnrs.get(nodeId); MvccSnapshotResponseListener lsnr; + Map map = snapLsnrs.get(nodeId); + + MvccSnapshotResponseListener lsnr; if (map != null && (lsnr = map.remove(msg.futureId())) != null) lsnr.onResponse(msg); @@ -2007,18 +2009,22 @@ private VacuumMetrics processPartition(VacuumTask task) throws IgniteCheckedExce try { GridCursor cursor = part.dataStore().cursor(KEY_ONLY); - KeyCacheObject prevKey = null; Object rest = null; + KeyCacheObject prevKey = null; + + Object rest = null; List cleanupRows = null; MvccSnapshot snapshot = task.snapshot(); - GridCacheContext cctx = null; int curCacheId = CU.UNDEFINED_CACHE_ID; + GridCacheContext cctx = null; + + int curCacheId = CU.UNDEFINED_CACHE_ID; boolean shared = part.group().sharedGroup(); - if (!shared) - cctx = part.group().singleCacheContext(); + if (!shared && (cctx = F.first(part.group().caches())) == null) + return metrics; while (cursor.next()) { if (isCancelled()) @@ -2032,19 +2038,27 @@ private VacuumMetrics processPartition(VacuumTask task) throws IgniteCheckedExce if (cctx == null) { assert shared; - curCacheId = row.cacheId(); - cctx = part.group().shared().cacheContext(curCacheId); + cctx = part.group().shared().cacheContext(curCacheId = row.cacheId()); + + if (cctx == null) + return metrics; } if (!prevKey.equals(row.key()) || (shared && curCacheId != row.cacheId())) { if (rest != null || !F.isEmpty(cleanupRows)) cleanup(part, prevKey, cleanupRows, rest, cctx, metrics); - cleanupRows = null; rest = null; + cleanupRows = null; - if (shared && curCacheId != row.cacheId()) + rest = null; + + if (shared && curCacheId != row.cacheId()) { cctx = part.group().shared().cacheContext(curCacheId = row.cacheId()); + if (cctx == null) + return metrics; + } + prevKey = row.key(); } @@ -2127,7 +2141,6 @@ private boolean actualize(GridCacheContext cctx, MvccDataRow row, } /** - * * @param part Local partition. * @param key Key. * @param cleanupRows Cleanup rows. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java index 33f457db5f231..c75393e7d1bf8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java @@ -758,12 +758,7 @@ public static boolean mvccEnabled(GridKernalContext ctx) { if (tx == null) tracker = new MvccQueryTrackerImpl(cctx); else if ((tracker = tx.mvccQueryTracker()) == null) - tracker = new StaticMvccQueryTracker(cctx, requestSnapshot(cctx, tx)) { - @Override public void onDone() { - // TODO IGNITE-8841 - checkActive(tx); - } - }; + tracker = new StaticMvccQueryTracker(cctx, requestSnapshot(cctx, tx)); if (tracker.snapshot() == null) // TODO IGNITE-7388 @@ -780,7 +775,9 @@ else if ((tracker = tx.mvccQueryTracker()) == null) */ public static MvccSnapshot requestSnapshot(GridCacheContext cctx, GridNearTxLocal tx) throws IgniteCheckedException { - MvccSnapshot snapshot; tx = checkActive(tx); + MvccSnapshot snapshot; + + tx = checkActive(tx); if ((snapshot = tx.mvccSnapshot()) == null) { MvccProcessor prc = cctx.shared().coordinators(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java index f21a22f8214fc..07aea4c57c67f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; @@ -540,8 +541,19 @@ private CacheQueryFuture execute0(@Nullable IgniteReducer rmtReduce MvccQueryTracker mvccTracker = null; - if (cctx.mvccEnabled() && mvccSnapshot == null) - mvccSnapshot = (mvccTracker = MvccUtils.mvccTracker(cctx, false)).snapshot(); + if (cctx.mvccEnabled() && mvccSnapshot == null) { + GridNearTxLocal tx = cctx.tm().userTx(); + + if (tx != null) + mvccSnapshot = MvccUtils.requestSnapshot(cctx, tx); + else { + mvccTracker = MvccUtils.mvccTracker(cctx, null); + + mvccSnapshot = mvccTracker.snapshot(); + } + + assert mvccSnapshot != null; + } boolean loc = nodes.size() == 1 && F.first(nodes).id().equals(cctx.localNodeId()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java index e51b7d04203f9..b2cbe0547a8c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractBasicCoordinatorFailoverTest.java @@ -106,10 +106,6 @@ protected void coordinatorFailureSimple( } catch (ClusterTopologyException e) { info("Expected exception: " + e); - - assertNotNull(e.retryReadyFuture()); - - e.retryReadyFuture().get(); } catch (CacheException e) { info("Expected exception: " + e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java index f5172c864b4fa..3ff88467a4b18 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractFeatureTest.java @@ -69,15 +69,22 @@ protected CacheMode cacheMode() { @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); + cleanPersistenceDir(); + startGrids(4); node = grid(0); } /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - super.beforeTest(); + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); @@ -93,8 +100,6 @@ protected CacheMode cacheMode() { /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { node.destroyCache(CACHE_NAME); - - super.afterTest(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java index 1abb45f5fd8ed..6e22f44108700 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java @@ -93,9 +93,11 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SCAN; import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL; import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.ReadMode.SQL_SUM; import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.DML; +import static org.apache.ignite.internal.processors.cache.mvcc.CacheMvccAbstractTest.WriteMode.PUT; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -868,6 +870,9 @@ protected void putAllGetAll( ReadMode readMode, WriteMode writeMode ) throws Exception { + if(readMode == SCAN && writeMode == PUT) + fail("https://issues.apache.org/jira/browse/IGNITE-7764"); + final int RANGE = 20; final int writers = 4; @@ -1052,6 +1057,9 @@ protected void updateNObjectsTest( ) throws Exception { + if(readMode == SCAN && writeMode == PUT) + fail("https://issues.apache.org/jira/browse/IGNITE-7764"); + final int TOTAL = 20; assert N <= TOTAL; @@ -1501,7 +1509,7 @@ final void verifyCoordinatorInternalState() throws Exception { * * @throws Exception If failed. */ - private void verifyOldVersionsCleaned() throws Exception { + protected void verifyOldVersionsCleaned() throws Exception { runVacuumSync(); // Check versions. @@ -1531,8 +1539,8 @@ private boolean checkOldVersions(boolean failIfNotCleaned) throws IgniteCheckedE if (!cctx.userCache() || !cctx.group().mvccEnabled()) continue; - for (Object e : cache.withKeepBinary()) { - IgniteBiTuple entry = (IgniteBiTuple)e; + for (Iterator it = cache.withKeepBinary().iterator(); it.hasNext(); ) { + IgniteBiTuple entry = (IgniteBiTuple)it.next(); KeyCacheObject key = cctx.toCacheKeyObject(entry.getKey()); @@ -1541,9 +1549,12 @@ private boolean checkOldVersions(boolean failIfNotCleaned) throws IgniteCheckedE if (vers.size() > 1) { if (failIfNotCleaned) - fail("[key=" + key.value(null, false) + "; vers=" + vers + ']'); - else + fail("[key=" + key.value(null, false) + "; vers=" + vers + ']'); + else { + U.closeQuiet((AutoCloseable)it); + return false; + } } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java index f9ab7bcfb208a..76a86048a059d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java @@ -74,25 +74,29 @@ public class CacheMvccClusterRestartTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { - cleanPersistenceDir(); - super.afterTestsStopped(); + + stopAllGrids(); + + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - super.beforeTest(); + fail("https://issues.apache.org/jira/browse/IGNITE-9394"); cleanPersistenceDir(); + + super.beforeTest(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - cleanPersistenceDir(); + super.afterTest(); stopAllGrids(); - super.afterTest(); + cleanPersistenceDir(); } /** @@ -124,8 +128,6 @@ public void testRestart3() throws Exception { private void restart1(int srvBefore, int srvAfter) throws Exception { Ignite srv0 = startGridsMultiThreaded(srvBefore); - srv0.active(true); - IgniteCache cache = srv0.createCache(cacheConfiguration()); Set keys = new HashSet<>(primaryKeys(cache, 1, 0)); @@ -141,8 +143,6 @@ private void restart1(int srvBefore, int srvAfter) throws Exception { srv0 = startGridsMultiThreaded(srvAfter); - srv0.active(true); - cache = srv0.cache(DEFAULT_CACHE_NAME); Map res = cache.getAll(keys); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java index 01268decca5c3..f1519dff46963 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java @@ -55,6 +55,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.KeyCacheObject; @@ -416,6 +417,8 @@ private void activeQueriesCleanup(final boolean tx) throws Exception { * @throws Exception If failed. */ public void testTxReadIsolationSimple() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-7764"); + Ignite srv0 = startGrids(4); client = true; @@ -2263,8 +2266,6 @@ public void testActiveQueryCleanupOnNodeFailure() throws Exception { checkActiveQueriesCleanup(ignite(0)); - verifyCoordinatorInternalState(); - try { fut.get(); } @@ -2466,10 +2467,6 @@ private void txPrepareFailureSimple( } catch (ClusterTopologyException e) { info("Expected exception: " + e); - - assertNotNull(e.retryReadyFuture()); - - e.retryReadyFuture().get(); } return null; @@ -2817,7 +2814,7 @@ public void testLoadWithStreamer() throws Exception { IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 2, 64)); - final int KEYS = 10_000; + final int KEYS = 1_000; Map data = new HashMap<>(); @@ -2905,6 +2902,8 @@ public void testUpdate_N_Objects_ClientServer_Backups2_Get() throws Exception { * @throws Exception If failed. */ public void testUpdate_N_Objects_ClientServer_Backups1_Scan() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-7764"); + int[] nValues = {3, 5, 10}; for (int n : nValues) { @@ -3262,7 +3261,9 @@ public void testInternalApi() throws Exception { CacheDataRow row = cctx.offheap().read(cctx, key0); - checkRow(cctx, row, key0, vers.get(0).get1()); + Object val = ((CacheObject)vers.get(0).get1()).value(cctx.cacheObjectContext(), false); + + checkRow(cctx, row, key0, val); for (IgniteBiTuple ver : vers) { MvccVersion cntr = ver.get2(); @@ -3272,18 +3273,20 @@ public void testInternalApi() throws Exception { row = cctx.offheap().mvccRead(cctx, key0, readVer); - checkRow(cctx, row, key0, ver.get1()); + Object verVal = ((CacheObject)ver.get1()).value(cctx.cacheObjectContext(), false); + + checkRow(cctx, row, key0, verVal); } checkRow(cctx, cctx.offheap().mvccRead(cctx, key0, version(vers.get(0).get2().coordinatorVersion() + 1, 1)), key0, - vers.get(0).get1()); + val); checkRow(cctx, cctx.offheap().mvccRead(cctx, key0, version(vers.get(0).get2().coordinatorVersion(), vers.get(0).get2().counter() + 1)), key0, - vers.get(0).get1()); + val); MvccSnapshotResponse ver = version(vers.get(0).get2().coordinatorVersion(), 100000); @@ -3296,8 +3299,11 @@ public void testInternalApi() throws Exception { if (v == vers.size() - 1) assertNull(row); - else - checkRow(cctx, row, key0, vers.get(v + 1).get1()); + else { + Object nextVal = ((CacheObject)vers.get(v + 1).get1()).value(cctx.cacheObjectContext(), false); + + checkRow(cctx, row, key0, nextVal); + } } } @@ -3316,7 +3322,8 @@ public void testInternalApi() throws Exception { * @throws Exception If failed. */ public void testExpiration() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-7956"); + fail("https://issues.apache.org/jira/browse/IGNITE-7311"); + final IgniteEx node = startGrid(0); IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, 64)); @@ -3370,6 +3377,8 @@ public void testExpiration() throws Exception { * @throws Exception If failed. */ public void testChangeExpireTime() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-7311"); + final IgniteEx node = startGrid(0); IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, 64)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java new file mode 100644 index 0000000000000..c1af42b6605ef --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java @@ -0,0 +1,70 @@ +/* + * 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.datastreamer; + +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; + +/** + * Check DataStreamer with Mvcc enabled. + */ +public class DataStreamProcessorMvccSelfTest extends DataStreamProcessorSelfTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration igniteConfiguration = super.getConfiguration(igniteInstanceName); + + CacheConfiguration[] cacheConfigurations = igniteConfiguration.getCacheConfiguration(); + + assert cacheConfigurations == null || cacheConfigurations.length == 0 + || (cacheConfigurations.length == 1 && cacheConfigurations[0].getAtomicityMode() == TRANSACTIONAL); + + igniteConfiguration.setMvccEnabled(true); + + return igniteConfiguration; + } + + /** {@inheritDoc} */ + @Override public void testPartitioned() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8149"); + + super.testPartitioned(); + } + + /** {@inheritDoc} */ + @Override public void testColocated() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8149"); + + super.testColocated(); + } + + /** {@inheritDoc} */ + @Override public void testReplicated() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8149"); + + super.testReplicated(); + } + + /** {@inheritDoc} */ + @Override public void testUpdateStore() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8582"); + + super.testUpdateStore(); + } +} 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 55086f3cf5a80..273456a1e0044 100755 --- 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 @@ -83,7 +83,6 @@ 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.colocated.GridDhtColocatedCache; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; import org.apache.ignite.internal.processors.cache.local.GridLocalCache; @@ -91,10 +90,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2; -import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; -import org.apache.ignite.internal.processors.cache.verify.PartitionKey; -import org.apache.ignite.internal.processors.cache.verify.VerifyBackupPartitionsTask; -import org.apache.ignite.internal.util.lang.GridAbsClosure; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; @@ -102,13 +97,10 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteBiInClosure; -import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.internal.visor.VisorTaskArgument; -import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTask; import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskArg; -import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskResult; import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskV2; +import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; @@ -128,7 +120,6 @@ import static org.apache.ignite.cache.CacheMode.LOCAL; import static org.apache.ignite.cache.CacheRebalanceMode.NONE; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled; -import static org.apache.ignite.internal.processors.cache.GridCacheUtils.retryTopologySafe; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java index b5275b9dba833..f87b14d22e2f3 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java @@ -30,7 +30,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentTransactionTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTransactionsTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccVacuumTest; -import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorMvccSeflTest; +import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorMvccSelfTest; /** * @@ -42,19 +42,25 @@ public class IgniteCacheMvccTestSuite extends TestSuite { public static TestSuite suite() { TestSuite suite = new TestSuite("IgniteCache MVCC Test Suite"); + // Basic tests. suite.addTestSuite(CacheMvccTransactionsTest.class); suite.addTestSuite(CacheMvccProcessorTest.class); - suite.addTestSuite(CacheMvccClusterRestartTest.class); + suite.addTestSuite(CacheMvccVacuumTest.class); suite.addTestSuite(CacheMvccConfigurationValidationTest.class); + + suite.addTestSuite(DataStreamProcessorMvccSelfTest.class); suite.addTestSuite(CacheMvccOperationChecksTest.class); + + // Concurrent ops tests. suite.addTestSuite(CacheMvccIteratorWithConcurrentTransactionTest.class); suite.addTestSuite(CacheMvccLocalEntriesWithConcurrentTransactionTest.class); suite.addTestSuite(CacheMvccScanQueryWithConcurrentTransactionTest.class); suite.addTestSuite(CacheMvccSizeWithConcurrentTransactionTest.class); - suite.addTestSuite(CacheMvccVacuumTest.class); + + // Failover tests. + suite.addTestSuite(CacheMvccClusterRestartTest.class); suite.addTestSuite(CacheMvccPartitionedCoordinatorFailoverTest.class); suite.addTestSuite(CacheMvccReplicatedCoordinatorFailoverTest.class); - suite.addTestSuite(DataStreamProcessorMvccSeflTest.class); return suite; } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java index 05df754394236..e9f293c789a91 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2FieldsIterator.java @@ -57,7 +57,7 @@ public H2FieldsIterator(ResultSet data, MvccQueryTracker mvccTracker, boolean fo } /** {@inheritDoc} */ - @Override public void onClose() { + @Override public void onClose() throws IgniteCheckedException { try { super.onClose(); } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java index e84ca043d5b84..814e83d2962cd 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2ResultSetIterator.java @@ -26,7 +26,6 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; import org.h2.jdbc.JdbcResultSet; import org.h2.result.ResultInterface; import org.h2.value.Value; @@ -99,13 +98,13 @@ protected H2ResultSetIterator(ResultSet data, boolean forUpdate) throws IgniteCh /** * @return {@code true} If next row was fetched successfully. */ - private boolean fetchNext() { + private boolean fetchNext() throws IgniteCheckedException { if (data == null) return false; try { - if (!data.next()){ - onClose(); + if (!data.next()) { + close(); return false; } @@ -138,7 +137,7 @@ private boolean fetchNext() { } /** {@inheritDoc} */ - @Override public boolean onHasNext() { + @Override public boolean onHasNext() throws IgniteCheckedException { return hasRow || (hasRow = fetchNext()); } @@ -164,15 +163,21 @@ private boolean fetchNext() { } /** {@inheritDoc} */ - @Override public void onClose(){ + @Override public void onClose() throws IgniteCheckedException { if (data == null) // Nothing to close. return; - U.closeQuiet(data); - - res = null; - data = null; + try { + data.close(); + } + catch (SQLException e) { + throw new IgniteSQLException(e); + } + finally { + res = null; + data = null; + } } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java new file mode 100644 index 0000000000000..8b3fbe35ad9fc --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.index; + +import java.util.concurrent.Callable; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * + */ +public class SqlTransactionsComandsSelfTest extends AbstractSchemaSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(commonConfiguration(0)); + + super.execute(grid(0), "CREATE TABLE INTS(k int primary key, v int) WITH \"wrap_value=false,cache_name=ints," + + "atomicity=transactional\""); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + + /** + * @throws Exception if failed. + */ + public void testBeginWithMvccDisabledThrows() throws Exception { + checkMvccDisabledBehavior("BEGIN"); + } + + /** + * @throws Exception if failed. + */ + public void testCommitWithMvccDisabledThrows() throws Exception { + checkMvccDisabledBehavior("COMMIT"); + } + + /** + * @throws Exception if failed. + */ + public void testRollbackWithMvccDisabledThrows() throws Exception { + checkMvccDisabledBehavior("rollback"); + } + + /** + * @param sql Operation to test. + * @throws Exception if failed. + */ + private void checkMvccDisabledBehavior(String sql) throws Exception { + try (IgniteEx node = startGrid(commonConfiguration(1))) { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + execute(node, sql); + + return null; + } + }, IgniteSQLException.class, "MVCC must be enabled in order to invoke transactional operation: " + sql); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsCommandsWithMvccEnabledSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsCommandsWithMvccEnabledSelfTest.java new file mode 100644 index 0000000000000..55dd59af0e5a6 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsCommandsWithMvccEnabledSelfTest.java @@ -0,0 +1,420 @@ +/* + * 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.index; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +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.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.GatewayProtectedCacheProxy; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionState; + +/** + * Tests to check behavior regarding transactions started via SQL. + */ +public class SqlTransactionsCommandsWithMvccEnabledSelfTest extends AbstractSchemaSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(commonConfiguration(0).setMvccEnabled(true)); + + super.execute(node(), "CREATE TABLE INTS(k int primary key, v int) WITH \"wrap_value=false,cache_name=ints," + + "atomicity=transactional\""); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * Test that BEGIN opens a transaction. + */ + public void testBegin() { + execute(node(), "BEGIN"); + + assertTxPresent(); + + assertTxState(tx(), TransactionState.ACTIVE); + } + + /** + * Test that COMMIT commits a transaction. + */ + public void testCommit() { + execute(node(), "BEGIN WORK"); + + assertTxPresent(); + + Transaction tx = tx(); + + assertTxState(tx, TransactionState.ACTIVE); + + execute(node(), "COMMIT TRANSACTION"); + + assertTxState(tx, TransactionState.COMMITTED); + + assertSqlTxNotPresent(); + } + + /** + * Test that COMMIT without a transaction yields nothing. + */ + public void testCommitNoTransaction() { + execute(node(), "COMMIT"); + } + + /** + * Test that ROLLBACK without a transaction yields nothing. + */ + public void testRollbackNoTransaction() { + execute(node(), "ROLLBACK"); + } + + /** + * Test that ROLLBACK rolls back a transaction. + */ + public void testRollback() { + execute(node(), "BEGIN TRANSACTION"); + + assertTxPresent(); + + Transaction tx = tx(); + + assertTxState(tx, TransactionState.ACTIVE); + + execute(node(), "ROLLBACK TRANSACTION"); + + assertTxState(tx, TransactionState.ROLLED_BACK); + + assertSqlTxNotPresent(); + } + + /** + * Test that attempting to perform various SQL operations within non SQL transaction yields an exception. + */ + public void testSqlOperationsWithinNonSqlTransaction() { + assertSqlOperationWithinNonSqlTransactionThrows("COMMIT"); + + assertSqlOperationWithinNonSqlTransactionThrows("ROLLBACK"); + + assertSqlOperationWithinNonSqlTransactionThrows("SELECT * from ints"); + + assertSqlOperationWithinNonSqlTransactionThrows("DELETE from ints"); + + assertSqlOperationWithinNonSqlTransactionThrows("INSERT INTO ints(k, v) values(10, 15)"); + + assertSqlOperationWithinNonSqlTransactionThrows("MERGE INTO ints(k, v) values(10, 15)"); + + assertSqlOperationWithinNonSqlTransactionThrows("UPDATE ints SET v = 100 WHERE k = 5"); + + assertSqlOperationWithinNonSqlTransactionThrows("create index idx on ints(v)"); + + assertSqlOperationWithinNonSqlTransactionThrows("CREATE TABLE T(k int primary key, v int)"); + } + + /** + * Check that trying to run given SQL statement both locally and in distributed mode yields an exception + * if transaction already has been marked as being of SQL type. + * @param sql SQL statement. + */ + private void assertSqlOperationWithinNonSqlTransactionThrows(final String sql) { + try (Transaction ignored = node().transactions().txStart()) { + node().cache("ints").put(1, 1); + + assertSqlException(new RunnableX() { + @Override public void run() throws Exception { + execute(node(), sql); + } + }, IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH); + } + + try (Transaction ignored = node().transactions().txStart()) { + node().cache("ints").put(1, 1); + + assertSqlException(new RunnableX() { + @Override public void run() throws Exception { + node().cache("ints").query(new SqlFieldsQuery(sql).setLocal(true)).getAll(); + } + }, IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH); + } + } + + /** + * Test that attempting to perform a cache API operation from within an SQL transaction fails. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + private void checkCacheOperationThrows(final String opName, final Object... args) { + execute(node(), "BEGIN"); + + try { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + try { + // We need to detect types based on arguments due to multiple overloads. + Class[] types; + + if (F.isEmpty(args)) + types = (Class[]) X.EMPTY_OBJECT_ARRAY; + else { + types = new Class[args.length]; + + for (int i = 0; i < args.length; i++) + types[i] = argTypeForObject(args[i]); + } + + Object res = U.invoke(GatewayProtectedCacheProxy.class, node().cache("ints"), + opName, types, args); + + if (opName.endsWith("Async")) + ((IgniteFuture)res).get(); + } + catch (IgniteCheckedException e) { + if (e.getCause() != null) { + try { + if (e.getCause().getCause() != null) + throw (Exception)e.getCause().getCause(); + else + fail(); + } + catch (IgniteException e1) { + // Some public API methods don't have IgniteCheckedException on their signature + // and thus may wrap it into an IgniteException. + if (e1.getCause() != null) + throw (Exception)e1.getCause(); + else + fail(); + } + } + else + fail(); + } + + return null; + } + }, IgniteCheckedException.class, + "SQL queries and cache operations may not be used in the same transaction."); + } + finally { + try { + execute(node(), "ROLLBACK"); + } + catch (Throwable e) { + // No-op. + } + } + } + + /** + * + */ + private static Class argTypeForObject(Object arg) { + if (arg instanceof Set) + return Set.class; + else if (arg instanceof Map) + return Map.class; + else if (arg.getClass().getName().startsWith("java.lang.")) + return Object.class; + else if (arg instanceof CacheEntryProcessor) + return CacheEntryProcessor.class; + else if (arg instanceof EntryProcessor) + return EntryProcessor.class; + else + return arg.getClass(); + } + + /** + * Test that attempting to perform a cache PUT operation from within an SQL transaction fails. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testCacheOperationsFromSqlTransaction() { + checkCacheOperationThrows("get", 1); + + checkCacheOperationThrows("getAsync", 1); + + checkCacheOperationThrows("getEntry", 1); + + checkCacheOperationThrows("getEntryAsync", 1); + + checkCacheOperationThrows("getAndPut", 1, 1); + + checkCacheOperationThrows("getAndPutAsync", 1, 1); + + checkCacheOperationThrows("getAndPutIfAbsent", 1, 1); + + checkCacheOperationThrows("getAndPutIfAbsentAsync", 1, 1); + + checkCacheOperationThrows("getAndReplace", 1, 1); + + checkCacheOperationThrows("getAndReplaceAsync", 1, 1); + + checkCacheOperationThrows("getAndRemove", 1); + + checkCacheOperationThrows("getAndRemoveAsync", 1); + + checkCacheOperationThrows("containsKey", 1); + + checkCacheOperationThrows("containsKeyAsync", 1); + + checkCacheOperationThrows("put", 1, 1); + + checkCacheOperationThrows("putAsync", 1, 1); + + checkCacheOperationThrows("putIfAbsent", 1, 1); + + checkCacheOperationThrows("putIfAbsentAsync", 1, 1); + + checkCacheOperationThrows("remove", 1); + + checkCacheOperationThrows("removeAsync", 1); + + checkCacheOperationThrows("remove", 1, 1); + + checkCacheOperationThrows("removeAsync", 1, 1); + + checkCacheOperationThrows("replace", 1, 1); + + checkCacheOperationThrows("replaceAsync", 1, 1); + + checkCacheOperationThrows("replace", 1, 1, 1); + + checkCacheOperationThrows("replaceAsync", 1, 1, 1); + + checkCacheOperationThrows("getAll", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("containsKeys", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("getEntries", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("putAll", Collections.singletonMap(1, 1)); + + checkCacheOperationThrows("removeAll", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("getAllAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("containsKeysAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("getEntriesAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("putAllAsync", Collections.singletonMap(1, 1)); + + checkCacheOperationThrows("removeAllAsync", new HashSet<>(Arrays.asList(1, 2))); + + checkCacheOperationThrows("invoke", 1, ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invoke", 1, CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAsync", 1, ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAsync", 1, CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAll", Collections.singletonMap(1, CACHE_ENTRY_PROC), X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAll", Collections.singleton(1), CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAll", Collections.singleton(1), ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAllAsync", Collections.singletonMap(1, CACHE_ENTRY_PROC), + X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAllAsync", Collections.singleton(1), CACHE_ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + + checkCacheOperationThrows("invokeAllAsync", Collections.singleton(1), ENTRY_PROC, X.EMPTY_OBJECT_ARRAY); + } + + /** */ + private final static EntryProcessor ENTRY_PROC = + new EntryProcessor() { + @Override public Object process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + return null; + } + }; + + /** */ + private final static CacheEntryProcessor CACHE_ENTRY_PROC = + new CacheEntryProcessor() { + @Override public Object process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + return null; + } + }; + + /** + * @return Node. + */ + private IgniteEx node() { + return grid(0); + } + + /** + * @return Currently open transaction. + */ + private Transaction tx() { + return node().transactions().tx(); + } + + /** + * Check that there's an open transaction with SQL flag. + */ + private void assertTxPresent() { + assertNotNull(tx()); + } + + /** {@inheritDoc} */ + @Override protected List> execute(Ignite node, String sql) { + return node.cache("ints").query(new SqlFieldsQuery(sql).setSchema(QueryUtils.DFLT_SCHEMA)).getAll(); + } + + /** + * Check that there's no open transaction. + */ + private void assertSqlTxNotPresent() { + assertNull(tx()); + } + + /** + * Check transaction state. + */ + private static void assertTxState(Transaction tx, TransactionState state) { + assertEquals(state, tx.state()); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java index 71e004bfbd8f5..998cb766668c7 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBackupsAbstractTest.java @@ -71,7 +71,7 @@ public abstract class CacheMvccBackupsAbstractTest extends CacheMvccAbstractTest public void testBackupsCoherenceSimple() throws Exception { disableScheduledVacuum = true; - ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, DFLT_PARTITION_COUNT) + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 2, 10) .setIndexedTypes(Integer.class, Integer.class); final int KEYS_CNT = 5_000; @@ -184,10 +184,10 @@ public void testBackupsCoherenceSimple() throws Exception { public void testBackupsCoherenceWithLargeOperations() throws Exception { disableScheduledVacuum = true; - ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 1, DFLT_PARTITION_COUNT) + ccfg = cacheConfiguration(cacheMode(), FULL_SYNC, 1, 10) .setIndexedTypes(Integer.class, Integer.class); - final int KEYS_CNT = 50_000; + final int KEYS_CNT = 5_000; assert KEYS_CNT % 2 == 0; startGrids(2); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java index 4ad667b9defb3..9e0b02f3335d5 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesAbstractTest.java @@ -629,6 +629,8 @@ public void testJoinTransactional_DistributedJoins_ClientServer2() throws Except * @throws Exception If failed. */ public void testDistributedJoinSimple() throws Exception { + disableScheduledVacuum = true; //TODO: IGNITE-9446: remove this after races in vacuum will be fixed. + startGridsMultiThreaded(4); Ignite srv0 = ignite(0); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java index ade37638b9ec2..69cf10883e5fb 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java @@ -634,7 +634,7 @@ public void testQueryReducerInsertVersionConflict() throws Exception { barrier.await(); String sqlText = "UPDATE MvccTestSqlIndexValue t SET idxVal1=" + - "(SELECT _val FROM \"int\".Integer WHERE t._key = _key ORDER BY _key)"; + "(SELECT _val FROM \"int\".Integer WHERE _key >= 5 AND _key <= 5 ORDER BY _key) WHERE _key = 5"; qry = new SqlFieldsQuery(sqlText); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java index 57cee61ecd209..1a6cb554d7543 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java @@ -18,44 +18,30 @@ package org.apache.ignite.internal.processors.query.h2; import java.io.File; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Semaphore; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.index.DynamicIndexAbstractSelfTest; -import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.query.GridQueryProcessor; -import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; -import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl; import org.apache.ignite.internal.util.lang.GridCursor; -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; -import org.apache.ignite.lang.IgniteBiTuple; /** * Index rebuild after node restart test. */ public class GridIndexRebuildSelfTest extends DynamicIndexAbstractSelfTest { /** Data size. */ - private final static int AMOUNT = 10; + protected static final int AMOUNT = 300; /** Data size. */ - private final static String CACHE_NAME = "T"; + protected static final String CACHE_NAME = "T"; /** Test instance to allow interaction with static context. */ private static GridIndexRebuildSelfTest INSTANCE; @@ -63,14 +49,13 @@ public class GridIndexRebuildSelfTest extends DynamicIndexAbstractSelfTest { /** Latch to signal that rebuild may start. */ private final CountDownLatch rebuildLatch = new CountDownLatch(1); - /** Latch to signal that concurrent put may start. */ - private final Semaphore rebuildSemaphore = new Semaphore(1, true); - /** {@inheritDoc} */ @Override protected IgniteConfiguration commonConfiguration(int idx) throws Exception { IgniteConfiguration cfg = super.commonConfiguration(idx); - cfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration().setPersistenceEnabled(true); + cfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration() + .setMaxSize(300*1024L*1024L) + .setPersistenceEnabled(true); return cfg; } @@ -85,24 +70,25 @@ public class GridIndexRebuildSelfTest extends DynamicIndexAbstractSelfTest { INSTANCE = this; } - /** - * Do test with MVCC enabled. - */ - public void testMvccEnabled() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-7259"); - doTest(true); + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + + cleanPersistenceDir(); } - /** - * Do test with MVCC disabled. - */ - public void testMvccDisabled() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-7259"); - doTest(false); + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + cleanPersistenceDir(); } /** - * Do test.

    + * Do test. + *

    * Steps are as follows: *

      *
    • Put some data;
    • @@ -120,12 +106,11 @@ public void testMvccDisabled() throws Exception { * index rebuild for them has happened after put. *
    * - * - * @param mvccEnabled MVCC flag. + *

    * @throws Exception if failed. */ - private void doTest(boolean mvccEnabled) throws Exception { - IgniteEx srv = startServer(mvccEnabled); + public void testIndexRebuild() throws Exception { + IgniteEx srv = startServer(); execute(srv, "CREATE TABLE T(k int primary key, v int) WITH \"cache_name=T,wrap_value=false," + "atomicity=transactional\""); @@ -136,12 +121,9 @@ private void doTest(boolean mvccEnabled) throws Exception { assertNotNull(cc); - if (mvccEnabled) - lockVersion(srv); - putData(srv, false); - checkDataState(srv, mvccEnabled, false); + checkDataState(srv, false); File cacheWorkDir = ((FilePageStoreManager)cc.context().shared().pageStore()).cacheWorkDir(cc.configuration()); @@ -151,22 +133,22 @@ private void doTest(boolean mvccEnabled) throws Exception { assertTrue(U.delete(idxPath)); - srv = startServer(mvccEnabled); + srv = startServer(); putData(srv, true); - checkDataState(srv, mvccEnabled, true); + checkDataState(srv, true); } /** * Check versions presence in index tree. + * * @param srv Node. - * @param mvccEnabled MVCC flag. * @param afterRebuild Whether index rebuild has occurred. * @throws IgniteCheckedException if failed. */ @SuppressWarnings({"ConstantConditions", "unchecked"}) - private void checkDataState(IgniteEx srv, boolean mvccEnabled, boolean afterRebuild) throws IgniteCheckedException { + protected void checkDataState(IgniteEx srv, boolean afterRebuild) throws IgniteCheckedException { IgniteInternalCache icache = srv.cachex(CACHE_NAME); IgniteCache cache = srv.cache(CACHE_NAME); @@ -179,47 +161,23 @@ private void checkDataState(IgniteEx srv, boolean mvccEnabled, boolean afterRebu while (cur.next()) { CacheDataRow row = cur.get(); - int key = row.key().value(icache.context().cacheObjectContext(), false); - - if (mvccEnabled) { - List> vers = store.mvccFindAllVersions(icache.context(), row.key()); - - if (!afterRebuild || key <= AMOUNT / 2) - assertEquals(key, vers.size()); - else { - // For keys affected by concurrent put there are two versions - - // -1 (concurrent put mark) and newest restored value as long as put cleans obsolete versions. - assertEquals(2, vers.size()); - - assertEquals(-1, vers.get(0).getKey()); - assertEquals(key, vers.get(1).getKey()); - } - } - else { - if (!afterRebuild || key <= AMOUNT / 2) - assertEquals(key, cache.get(key)); - else - assertEquals(-1, cache.get(key)); - } + int key = row.key().value(icache.context().cacheObjectContext(), false); + + if (!afterRebuild || key <= AMOUNT / 2) + assertEquals(key, cache.get(key)); + else + assertEquals(-1, cache.get(key)); } } } - /** - * Lock coordinator version in order to keep MVCC versions in place. - * @param node Node. - * @throws IgniteCheckedException if failed. - */ - private static void lockVersion(IgniteEx node) throws IgniteCheckedException { - node.context().coordinators().requestSnapshotAsync().get(); - } - /** * Put data to cache. + * * @param node Node. * @throws Exception if failed. */ - private void putData(Ignite node, final boolean forConcurrentPut) throws Exception { + protected void putData(Ignite node, final boolean forConcurrentPut) throws Exception { final IgniteCache cache = node.cache(CACHE_NAME); assertNotNull(cache); @@ -230,13 +188,9 @@ private void putData(Ignite node, final boolean forConcurrentPut) throws Excepti if (i <= AMOUNT / 2) continue; - rebuildSemaphore.acquire(); - cache.put(i, -1); rebuildLatch.countDown(); - - rebuildSemaphore.release(); } else { // Data streamer is not used intentionally in order to preserve all versions. @@ -248,16 +202,16 @@ private void putData(Ignite node, final boolean forConcurrentPut) throws Excepti /** * Start server node. - * @param mvccEnabled MVCC flag. + * * @return Started node. * @throws Exception if failed. */ - private IgniteEx startServer(boolean mvccEnabled) throws Exception { + protected IgniteEx startServer() throws Exception { // Have to do this for each starting node - see GridQueryProcessor ctor, it nulls // idxCls static field on each call. GridQueryProcessor.idxCls = BlockingIndexing.class; - IgniteConfiguration cfg = serverConfiguration(0).setMvccEnabled(mvccEnabled); + IgniteConfiguration cfg = serverConfiguration(0); IgniteEx res = startGrid(cfg); @@ -266,21 +220,6 @@ private IgniteEx startServer(boolean mvccEnabled) throws Exception { return res; } - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - super.afterTestsStopped(); - - cleanPersistenceDir(); - } - /** * Blocking indexing processor. */ @@ -295,56 +234,7 @@ private static class BlockingIndexing extends IgniteH2Indexing { else firstRbld = false; - int cacheId = CU.cacheId(cacheName); - - GridCacheContext cctx = ctx.cache().context().cacheContext(cacheId); - - final GridCacheQueryManager qryMgr = cctx.queries(); - - SchemaIndexCacheVisitor visitor = new SchemaIndexCacheVisitorImpl(cctx); - - visitor.visit(new TestRebuildClosure(qryMgr, cctx.mvccEnabled())); - - for (H2TableDescriptor tblDesc : tables(cacheName)) - tblDesc.table().markRebuildFromHashInProgress(false); - } - } - - /** - * Test closure. - */ - private final static class TestRebuildClosure extends RebuildIndexFromHashClosure { - /** Seen keys set to track moment when concurrent put may start. */ - private final Set keys = - Collections.newSetFromMap(new ConcurrentHashMap()); - - /** - * @param qryMgr Query manager. - * @param mvccEnabled MVCC status flag. - */ - TestRebuildClosure(GridCacheQueryManager qryMgr, boolean mvccEnabled) { - super(qryMgr, mvccEnabled); - } - - /** {@inheritDoc} */ - @Override public synchronized void apply(CacheDataRow row) throws IgniteCheckedException { - // For half of the keys, we want to do rebuild - // after corresponding key had been put from a concurrent thread. - boolean keyFirstMet = keys.add(row.key()) && keys.size() > AMOUNT / 2; - - if (keyFirstMet) { - try { - INSTANCE.rebuildSemaphore.acquire(); - } - catch (InterruptedException e) { - throw new IgniteCheckedException(e); - } - } - - super.apply(row); - - if (keyFirstMet) - INSTANCE.rebuildSemaphore.release(); + super.rebuildIndexesFromHash(cacheName); } } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java new file mode 100644 index 0000000000000..3d65db8f58410 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.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.processors.query.h2; + +import java.io.File; +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; + +/** + * Index rebuild after node restart test. + */ +public class GridIndexRebuildWithMvccEnabledSelfTest extends GridIndexRebuildSelfTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration serverConfiguration(int idx, boolean filter) throws Exception { + return super.serverConfiguration(idx, filter) + .setMvccVacuumTimeInterval(Integer.MAX_VALUE) + .setMvccEnabled(true); + } + + /** {@inheritDoc} */ + public void testIndexRebuild() throws Exception { + IgniteEx srv = startServer(); + + execute(srv, "CREATE TABLE T(k int primary key, v int) WITH \"cache_name=T,wrap_value=false," + + "atomicity=transactional\""); + + execute(srv, "CREATE INDEX IDX ON T(v)"); + + IgniteInternalCache cc = srv.cachex(CACHE_NAME); + + assertNotNull(cc); + + lockVersion(srv); + + putData(srv, false); + + checkDataState(srv, false); + + File cacheWorkDir = ((FilePageStoreManager)cc.context().shared().pageStore()).cacheWorkDir(cc.configuration()); + + File idxPath = cacheWorkDir.toPath().resolve("index.bin").toFile(); + + stopAllGrids(); + + assertTrue(U.delete(idxPath)); + + srv = startServer(); + + putData(srv, true); + + checkDataState(srv, true); + } + + /** + * Lock coordinator version in order to keep MVCC versions in place. + * + * @param node Node. + * @throws IgniteCheckedException if failed. + */ + private static void lockVersion(IgniteEx node) throws IgniteCheckedException { + node.context().coordinators().requestSnapshotAsync().get(); + } + + /** {@inheritDoc} */ + protected void checkDataState(IgniteEx srv, boolean afterRebuild) throws IgniteCheckedException { + IgniteInternalCache icache = srv.cachex(CACHE_NAME); + + assertNotNull(icache); + + CacheObjectContext coCtx = icache.context().cacheObjectContext(); + + for (IgniteCacheOffheapManager.CacheDataStore store : icache.context().offheap().cacheDataStores()) { + GridCursor cur = store.cursor(); + + while (cur.next()) { + CacheDataRow row = cur.get(); + + int key = row.key().value(coCtx, false); + + List> vers = store.mvccFindAllVersions(icache.context(), row.key()); + + if (!afterRebuild || key <= AMOUNT / 2) + assertEquals(key, vers.size()); + else { + // For keys affected by concurrent put there are two versions - + // -1 (concurrent put mark) and newest restored value as long as put cleans obsolete versions. + assertEquals(2, vers.size()); + + Object val0 = ((CacheObject)vers.get(0).getKey()).value(coCtx, false); + Object val1 = ((CacheObject)vers.get(1).getKey()).value(coCtx, false); + + assertEquals(-1, val0); + assertEquals(key, val1); + } + + } + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java index a8087da077850..ab9d2e6763f3f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.index.SqlTransactionsCommandsWithMvccEnabledSelfTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccBulkLoadTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccDmlSimpleTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccIteratorWithConcurrentJdbcTransactionTest; @@ -37,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccScanQueryWithConcurrentJdbcTransactionTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentJdbcTransactionTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccStreamingInsertTest; +import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildWithMvccEnabledSelfTest; /** * @@ -48,25 +50,39 @@ public class IgniteCacheMvccSqlTestSuite extends TestSuite { public static TestSuite suite() { TestSuite suite = new TestSuite("IgniteCache SQL MVCC Test Suite"); + // Simle tests. + suite.addTestSuite(CacheMvccDmlSimpleTest.class); + suite.addTestSuite(SqlTransactionsCommandsWithMvccEnabledSelfTest.class); + + suite.addTestSuite(GridIndexRebuildWithMvccEnabledSelfTest.class); + + // JDBC tests. suite.addTestSuite(CacheMvccSizeWithConcurrentJdbcTransactionTest.class); suite.addTestSuite(CacheMvccScanQueryWithConcurrentJdbcTransactionTest.class); suite.addTestSuite(CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest.class); suite.addTestSuite(CacheMvccIteratorWithConcurrentJdbcTransactionTest.class); + + // Load tests. + suite.addTestSuite(CacheMvccBulkLoadTest.class); + suite.addTestSuite(CacheMvccStreamingInsertTest.class); + suite.addTestSuite(CacheMvccPartitionedSqlQueriesTest.class); suite.addTestSuite(CacheMvccReplicatedSqlQueriesTest.class); suite.addTestSuite(CacheMvccPartitionedSqlTxQueriesTest.class); suite.addTestSuite(CacheMvccReplicatedSqlTxQueriesTest.class); + suite.addTestSuite(CacheMvccPartitionedSqlTxQueriesWithReducerTest.class); suite.addTestSuite(CacheMvccReplicatedSqlTxQueriesWithReducerTest.class); suite.addTestSuite(CacheMvccPartitionedSelectForUpdateQueryTest.class); suite.addTestSuite(CacheMvccReplicatedSelectForUpdateQueryTest.class); + + // Failover tests. suite.addTestSuite(CacheMvccPartitionedBackupsTest.class); suite.addTestSuite(CacheMvccReplicatedBackupsTest.class); + suite.addTestSuite(CacheMvccPartitionedSqlCoordinatorFailoverTest.class); suite.addTestSuite(CacheMvccReplicatedSqlCoordinatorFailoverTest.class); - suite.addTestSuite(CacheMvccBulkLoadTest.class); - suite.addTestSuite(CacheMvccStreamingInsertTest.class); - suite.addTestSuite(CacheMvccDmlSimpleTest.class); + return suite; } 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 b0670e80d3187..671db13178b70 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 @@ -110,7 +110,6 @@ import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFIlterBasicSelfTest; import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFilterCoordinatorBasicSelfTest; import org.apache.ignite.internal.processors.cache.index.H2ConnectionLeaksSelfTest; -import org.apache.ignite.internal.processors.cache.index.IgniteDecimalSelfTest; import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsClientBasicSelfTest; import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerBasicSelfTest; import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerCoordinatorBasicSelfTest; @@ -131,11 +130,11 @@ import org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest; import org.apache.ignite.internal.processors.cache.index.H2RowCachePageEvictionTest; import org.apache.ignite.internal.processors.cache.index.H2RowCacheSelfTest; +import org.apache.ignite.internal.processors.cache.index.IgniteDecimalSelfTest; import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest; import org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest; import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest; -import org.apache.ignite.internal.processors.cache.index.SqlTransactionsComandsWithMvccDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.index.SqlTransactionsSelfTest; +import org.apache.ignite.internal.processors.cache.index.SqlTransactionsComandsSelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest; @@ -165,18 +164,18 @@ import org.apache.ignite.internal.processors.query.LazyQuerySelfTest; import org.apache.ignite.internal.processors.query.MultipleStatementsSqlQuerySelfTest; import org.apache.ignite.internal.processors.query.SqlIllegalSchemaSelfTest; -import org.apache.ignite.internal.processors.query.SqlSystemViewsSelfTest; import org.apache.ignite.internal.processors.query.SqlPushDownFunctionTest; import org.apache.ignite.internal.processors.query.SqlSchemaSelfTest; +import org.apache.ignite.internal.processors.query.SqlSystemViewsSelfTest; import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest; import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest; import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildSelfTest; -import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest; import org.apache.ignite.internal.processors.query.h2.H2ResultSetIteratorNullifyOnEndSelfTest; +import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest; import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest; import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest; -import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest; import org.apache.ignite.internal.processors.query.h2.PreparedStatementExSelfTest; +import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest; import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest; import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest; import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest; @@ -447,8 +446,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridIndexRebuildSelfTest.class); - suite.addTestSuite(SqlTransactionsSelfTest.class); - suite.addTestSuite(SqlTransactionsComandsWithMvccDisabledSelfTest.class); + suite.addTestSuite(SqlTransactionsComandsSelfTest.class); suite.addTestSuite(IgniteSqlDefaultValueTest.class); suite.addTestSuite(IgniteDecimalSelfTest.class); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs index 1fd8e72d92c42..1f600dda87fd8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs @@ -80,7 +80,10 @@ public class IgniteConfigurationParityTest "TimeServerPortBase", "TimeServerPortRange", "IncludeProperties", - "isAutoActivationEnabled" // IGNITE-7301 + "isAutoActivationEnabled", // IGNITE-7301 + "isMvccEnabled", //TODO: IGNITE-9390: Remove when Mvcc support will be added. + "MvccVacuumTimeInterval", //TODO: IGNITE-9390: Remove when Mvcc support will be added. + "MvccVacuumThreadCnt" //TODO: IGNITE-9390: Remove when Mvcc support will be added. }; /// From 17217c31b831fdd214f02fc44c6982977d2172d8 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Sat, 1 Sep 2018 23:17:27 +0300 Subject: [PATCH 32/95] Revert "IGNITE-9448 Update ZooKeeper version to 3.4.13 - Fixes #4661." This reverts commit 4d736fc20a6e667e841201ad2205f1440546191b. --- parent/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parent/pom.xml b/parent/pom.xml index f5b49ba39a8a4..76e39b79e29fa 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -122,7 +122,7 @@ 2.2.0 0.8.3 0.5 - 3.4.13 + 3.4.6 * From cf0215e37a7f6401f18612ff128cc8875d0263e0 Mon Sep 17 00:00:00 2001 From: Oleg Ignatenko Date: Mon, 3 Sep 2018 13:04:50 +0300 Subject: [PATCH 33/95] IGNITE-9348: ML examples improvements this closes #4662 --- examples/DEVNOTES.txt | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 examples/DEVNOTES.txt diff --git a/examples/DEVNOTES.txt b/examples/DEVNOTES.txt new file mode 100644 index 0000000000000..ba578bb55a666 --- /dev/null +++ b/examples/DEVNOTES.txt @@ -0,0 +1,11 @@ +Ignite Examples Build Instructions +================================== +1) Compile and install Ignite from project root folder: + + mvn clean install -Pall-java,all-scala,licenses -DskipTests + + (If needed, refer DEVNOTES.txt in project root folder for most up-to-date build instructions.) + +2) Build examples from "examples" sub-folder under Ignite project root: + + mvn clean package -DskipTests From d59db34060f330659088dab426ae992fc960ef35 Mon Sep 17 00:00:00 2001 From: Evgeny Stanilovskiy Date: Mon, 3 Sep 2018 16:21:58 +0300 Subject: [PATCH 34/95] IGNITE-584 Fixed scan query to return consistent results on changing topology - Fixes #4351. Signed-off-by: Alexey Goncharuk --- .../ignite/internal/processors/cache/GridCacheMapEntry.java | 2 +- .../processors/datastructures/GridSetQueryPredicate.java | 4 ++-- .../cache/datastructures/GridCacheSetAbstractSelfTest.java | 4 ---- .../GridCacheSetFailoverAbstractSelfTest.java | 6 ++---- .../GridCachePartitionedAtomicSetFailoverSelfTest.java | 2 +- .../partitioned/IgnitePartitionedSetNoBackupsSelfTest.java | 2 +- 6 files changed, 7 insertions(+), 13 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 4dc49441c2347..8c2b939bb5b5d 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 @@ -3357,7 +3357,7 @@ else if (deletedUnlocked()) null, topVer); - cctx.dataStructures().onEntryUpdated(key, false, true); + cctx.dataStructures().onEntryUpdated(key, false, false); } onUpdateFinished(updateCntr); 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 bc6c1827a0774..2de3dec775d06 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 @@ -99,7 +99,7 @@ public IgniteUuid setId() { */ private boolean filterKeys() { return !collocated && !(ctx.isLocal() || ctx.isReplicated()) && - (ctx.config().getBackups() > 0 || CU.isNearEnabled(ctx)); + (CU.isNearEnabled(ctx) || ctx.isPartitioned()); } /** {@inheritDoc} */ @@ -118,4 +118,4 @@ private boolean filterKeys() { @Override public String toString() { return S.toString(GridSetQueryPredicate.class, this); } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java index 9a707eb56726f..0ed4a973b1fdf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java @@ -582,8 +582,6 @@ public void testNodeJoinsAndLeaves() throws Exception { if (collectionCacheMode() == LOCAL) return; - fail("https://issues.apache.org/jira/browse/IGNITE-584"); - testNodeJoinsAndLeaves(false); } @@ -594,8 +592,6 @@ public void testNodeJoinsAndLeavesCollocated() throws Exception { if (collectionCacheMode() == LOCAL) return; - fail("https://issues.apache.org/jira/browse/IGNITE-584"); - testNodeJoinsAndLeaves(true); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java index f8af2a24e5205..ce320bd4a5ec6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java @@ -116,8 +116,7 @@ public void testNodeRestart() throws Exception { try { int size = set.size(); - // TODO: IGNITE-584, check for equality when IGNITE-584 fixed. - assertTrue(size > 0); + assertEquals(ITEMS, size); } catch (IgniteException ignore) { // No-op. @@ -134,8 +133,7 @@ public void testNodeRestart() throws Exception { cnt++; } - // TODO: IGNITE-584, check for equality when IGNITE-584 fixed. - assertTrue(cnt > 0); + assertEquals(ITEMS, cnt); } catch (IgniteException ignore) { // No-op. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSetFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSetFailoverSelfTest.java index c2af2b1897e6e..4673549ca88aa 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSetFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSetFailoverSelfTest.java @@ -35,4 +35,4 @@ public class GridCachePartitionedAtomicSetFailoverSelfTest extends GridCacheSetF @Override public void testNodeRestart() throws Exception { fail("https://issues.apache.org/jira/browse/IGNITE-170"); } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedSetNoBackupsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedSetNoBackupsSelfTest.java index 4daaeca01d243..5f09dfa84a54b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedSetNoBackupsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedSetNoBackupsSelfTest.java @@ -70,4 +70,4 @@ public void testCollocation() throws Exception { } } } -} \ No newline at end of file +} From da477f190247730acc5b00610835adb9c098673e Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Mon, 3 Sep 2018 16:50:05 +0300 Subject: [PATCH 35/95] IGNITE-8886: Binary: fixed footer offset calculation when raw data is present. This closes #4643. --- .../internal/binary/BinaryReaderExImpl.java | 2 +- .../binary/BinaryMarshallerSelfTest.java | 35 +++++++++++++++++-- 2 files changed, 33 insertions(+), 4 deletions(-) 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 ab1f874386dd9..38934f07d6ae0 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 @@ -240,7 +240,7 @@ public BinaryReaderExImpl(BinaryContext ctx, footerStart = start + offset; if (BinaryUtils.hasRaw(flags)) { - footerLen = len - offset - 4; + footerLen = len - offset; rawOff = start + in.readIntPositioned(start + len - 4); } else { 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 4a7d5d557ad84..81a087d0e894c 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 @@ -3061,6 +3061,33 @@ public void testUnregisteredClass() throws Exception { assertEquals(Value.class, res.cls); } + /** + * @throws Exception If failed. + */ + public void testMixedRawCollections() throws Exception { + Collection excludedClasses = Arrays.asList( + ObjectRaw.class.getName(), + ObjectWithRaw.class.getName(), + Value.class.getName()); + + BinaryMarshaller m0 = binaryMarshaller(null, excludedClasses); + BinaryMarshaller m1 = binaryMarshaller(); + + Value obj = new Value(27); + ObjectWithRaw objectWithRaw = new ObjectWithRaw(27, 13); + ObjectRaw objectRaw = new ObjectRaw(27, 13); + + Value objOther = new Value(26); + ObjectWithRaw objectWithRawOther = new ObjectWithRaw(26, 13); + ObjectRaw objectRawOther = new ObjectRaw(26, 13); + + ArrayList collection = new ArrayList(Arrays.asList( + obj, objectWithRawOther, objectRaw, objectWithRaw, objectRawOther, objOther)); + + marshalUnmarshal(collection, m0); + marshalUnmarshal(collection, m1); + } + /** * @throws Exception If failed. */ @@ -5473,8 +5500,10 @@ public ObjectRaw(int val0, int val1) { /** {@inheritDoc} */ @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { - val0 = reader.rawReader().readInt(); - val1 = reader.rawReader().readInt(); + BinaryRawReader rawReader = reader.rawReader(); + + val0 = rawReader.readInt(); + val1 = rawReader.readInt(); } } @@ -5592,4 +5621,4 @@ private Object readResolve() { } } } -} \ No newline at end of file +} From b51519ad8831ea7077954ad5b10057565b66e06b Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 3 Sep 2018 19:45:44 +0300 Subject: [PATCH 36/95] IGNITE-9425 Fixed NPE on index rebuild - Fixes #4647. Signed-off-by: Alexey Goncharuk --- .../GridCacheDatabaseSharedManager.java | 3 +- .../IgniteTwoRegionsRebuildIndexTest.java | 124 ++++++++++++++++++ .../IgnitePdsWithIndexingTestSuite.java | 2 + 3 files changed, 128 insertions(+), 1 deletion(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteTwoRegionsRebuildIndexTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 2e5f26fb4c4bb..a86b9f5c449f0 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -1362,7 +1362,8 @@ private void prepareIndexRebuildFuture(int cacheId) { final int cacheId = cacheCtx.cacheId(); final GridFutureAdapter usrFut = idxRebuildFuts.get(cacheId); - if (!cctx.pageStore().hasIndexStore(cacheCtx.groupId()) && cacheCtx.affinityNode()) { + if (!cctx.pageStore().hasIndexStore(cacheCtx.groupId()) && cacheCtx.affinityNode() + && cacheCtx.group().persistenceEnabled()) { IgniteInternalFuture rebuildFut = cctx.kernalContext().query() .rebuildIndexesFromHash(Collections.singleton(cacheCtx.cacheId())); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteTwoRegionsRebuildIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteTwoRegionsRebuildIndexTest.java new file mode 100644 index 0000000000000..7d5b296f931ca --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteTwoRegionsRebuildIndexTest.java @@ -0,0 +1,124 @@ +/* + * 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.database; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests the case when preformed index rebuild for created by client in-memory cache. + */ +public class IgniteTwoRegionsRebuildIndexTest extends GridCommonAbstractTest { + /** */ + private static final String PERSISTED_CACHE = "persisted"; + + /** */ + private static final String INMEMORY_CACHE = "inmemory"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + boolean client = igniteInstanceName.startsWith("client"); + + DataStorageConfiguration dsCfg = new DataStorageConfiguration(); + + if (!client) { + DataRegionConfiguration drCfg1 = new DataRegionConfiguration(); + drCfg1.setMaxSize(16 * 1024 * 1024); + drCfg1.setName("nopersistence"); + drCfg1.setInitialSize(drCfg1.getMaxSize()); + drCfg1.setPersistenceEnabled(false); + + DataRegionConfiguration drCfg2 = new DataRegionConfiguration(); + drCfg2.setMaxSize(16 * 1024 * 1024); + drCfg2.setName("persistence"); + drCfg2.setInitialSize(drCfg2.getMaxSize()); + drCfg2.setPersistenceEnabled(true); + + dsCfg.setDataRegionConfigurations(drCfg1, drCfg2); + + cfg.setDataStorageConfiguration(dsCfg); + } + else { + CacheConfiguration ccfg1 = new CacheConfiguration(PERSISTED_CACHE); + CacheConfiguration ccfg2 = new CacheConfiguration(INMEMORY_CACHE); + + ccfg1.setDataRegionName("persistence"); + ccfg2.setDataRegionName("nopersistence"); + + cfg.setCacheConfiguration(ccfg1, ccfg2); + cfg.setClientMode(true); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * @throws Exception If failed. + */ + public void testRebuildIndexes() throws Exception { + startGrid("server"); + Ignite client = startGrid("client"); + + client.cluster().active(true); + + populateData(client, PERSISTED_CACHE); + populateData(client, INMEMORY_CACHE); + + stopGrid("server"); + startGrid("server"); + + stopGrid("client"); + startGrid("client"); + } + + /** + * @param ignite Ignite. + * @param cacheName Cache name. + */ + private void populateData(Ignite ignite, String cacheName) { + try (IgniteDataStreamer streamer = ignite.dataStreamer(cacheName)) { + for (int i = 0; i < 1000; i++) + streamer.addData(i, i); + + streamer.flush(); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java index 033e5a9751308..67b9fad63cde2 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.database.IgniteDbSingleNodeWithIndexingWalRestoreTest; import org.apache.ignite.internal.processors.database.IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest; import org.apache.ignite.internal.processors.database.IgnitePersistentStoreSchemaLoadTest; +import org.apache.ignite.internal.processors.database.IgniteTwoRegionsRebuildIndexTest; /** * @@ -44,6 +45,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgnitePdsSingleNodeWithIndexingAndGroupPutGetPersistenceSelfTest.class); suite.addTestSuite(IgnitePersistentStoreSchemaLoadTest.class); suite.addTestSuite(IgnitePersistentStoreQueryWithMultipleClassesPerCacheTest.class); + suite.addTestSuite(IgniteTwoRegionsRebuildIndexTest.class); return suite; } From 3660ef284c7497c05c77e4720250487e7b903bbe Mon Sep 17 00:00:00 2001 From: shroman Date: Tue, 4 Sep 2018 12:32:32 +0900 Subject: [PATCH 37/95] IGNITE-8286: ScanQuery ignore setLocal with non local partition. - Fixes #3871. Signed-off-by: shroman --- .../cache/IgniteCacheProxyImpl.java | 3 +- .../cache/query/GridCacheQueryAdapter.java | 30 +++++++-- .../cache/query/GridCacheQueryManager.java | 8 ++- ...cheScanPartitionQueryFallbackSelfTest.java | 62 +++++++++++++++++++ 4 files changed, 95 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java index c21ad0bb87015..69ea562f6433e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java @@ -379,7 +379,8 @@ private QueryCursor query( IgniteBiPredicate p = scanQry.getFilter(); - final CacheQuery qry = ctx.queries().createScanQuery(p, transformer, scanQry.getPartition(), isKeepBinary); + final CacheQuery qry = ctx.queries().createScanQuery( + p, transformer, scanQry.getPartition(), isKeepBinary, scanQry.isLocal()); if (scanQry.getPageSize() > 0) qry.pageSize(scanQry.getPageSize()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java index 07aea4c57c67f..0e3ab43c88173 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java @@ -119,6 +119,9 @@ public class GridCacheQueryAdapter implements CacheQuery { /** */ private volatile boolean incBackups; + /** Local query. */ + private boolean forceLocal; + /** */ private volatile boolean dedup; @@ -143,13 +146,15 @@ public class GridCacheQueryAdapter implements CacheQuery { * @param filter Scan filter. * @param part Partition. * @param keepBinary Keep binary flag. + * @param forceLocal Flag to force local query. */ public GridCacheQueryAdapter(GridCacheContext cctx, GridCacheQueryType type, @Nullable IgniteBiPredicate filter, @Nullable IgniteClosure transform, @Nullable Integer part, - boolean keepBinary) { + boolean keepBinary, + boolean forceLocal) { assert cctx != null; assert type != null; assert part == null || part >= 0; @@ -160,6 +165,7 @@ public GridCacheQueryAdapter(GridCacheContext cctx, this.transform = transform; this.part = part; this.keepBinary = keepBinary; + this.forceLocal = forceLocal; log = cctx.logger(getClass()); @@ -312,6 +318,13 @@ public void keepBinary(boolean keepBinary) { this.keepBinary = keepBinary; } + /** + * @return {@code True} if the query is forced local. + */ + public boolean forceLocal() { + return forceLocal; + } + /** * @return Security subject ID. */ @@ -516,15 +529,20 @@ private CacheQueryFuture execute0(@Nullable IgniteReducer rmtReduce /** {@inheritDoc} */ @SuppressWarnings({"IfMayBeConditional", "unchecked"}) @Override public GridCloseableIterator executeScanQuery() throws IgniteCheckedException { - assert type == SCAN : "Wrong processing of qyery: " + type; + assert type == SCAN : "Wrong processing of query: " + type; // Affinity nodes snapshot. Collection nodes = new ArrayList<>(nodes()); cctx.checkSecurity(SecurityPermission.CACHE_READ); - if (nodes.isEmpty() && part == null) + if (nodes.isEmpty()) { + if (part != null && forceLocal) + throw new IgniteCheckedException("No queryable nodes for partition " + part + + " [forced local query=" + this + "]"); + return new GridEmptyCloseableIterator(); + } if (log.isDebugEnabled()) log.debug("Executing query [query=" + this + ", nodes=" + nodes + ']'); @@ -612,9 +630,10 @@ private Collection nodes() throws IgniteCheckedException { * @param cctx Cache context. * @param prj Projection (optional). * @return Collection of data nodes in provided projection (if any). + * @throws IgniteCheckedException If partition number is invalid. */ private static Collection nodes(final GridCacheContext cctx, - @Nullable final ClusterGroup prj, @Nullable final Integer part) { + @Nullable final ClusterGroup prj, @Nullable final Integer part) throws IgniteCheckedException { assert cctx != null; final AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion(); @@ -624,6 +643,9 @@ private static Collection nodes(final GridCacheContext cctx, if (prj == null && part == null) return affNodes; + if (part != null && part >= cctx.affinity().partitions()) + throw new IgniteCheckedException("Invalid partition number: " + part); + final Set owners = part == null ? Collections.emptySet() : new HashSet<>(cctx.topology().owners(part, topVer)); 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 c209602c62e67..281400e30e620 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 @@ -2723,7 +2723,7 @@ public CacheQuery createSpiQuery(boolean keepBinary) { */ public CacheQuery createScanQuery(@Nullable IgniteBiPredicate filter, @Nullable Integer part, boolean keepBinary) { - return createScanQuery(filter, null, part, keepBinary); + return createScanQuery(filter, null, part, keepBinary, false); } /** @@ -2733,18 +2733,20 @@ public CacheQuery createScanQuery(@Nullable IgniteBiPredicate filte * @param trans Transformer. * @param part Partition. * @param keepBinary Keep binary flag. + * @param forceLocal Flag to force local scan. * @return Created query. */ public CacheQuery createScanQuery(@Nullable IgniteBiPredicate filter, @Nullable IgniteClosure trans, - @Nullable Integer part, boolean keepBinary) { + @Nullable Integer part, boolean keepBinary, boolean forceLocal) { return new GridCacheQueryAdapter(cctx, SCAN, filter, trans, part, - keepBinary); + keepBinary, + forceLocal); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java index 999b1adcf3358..3afcad8816971 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java @@ -30,6 +30,7 @@ import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; @@ -57,6 +58,7 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; 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; /** @@ -148,6 +150,66 @@ public void testScanLocal() throws Exception { } } + /** + * Scan (with explicit {@code setLocal(true)}) should perform on the local node. + * + * @throws Exception If failed. + */ + public void testScanLocalExplicit() throws Exception { + cacheMode = CacheMode.PARTITIONED; + backups = 0; + commSpiFactory = new TestLocalCommunicationSpiFactory(); + + try { + Ignite ignite = startGrids(GRID_CNT); + + IgniteCacheProxy cache = fillCache(ignite); + + int part = anyLocalPartition(cache.context()); + + QueryCursor> qry = + cache.query(new ScanQuery().setPartition(part).setLocal(true)); + + doTestScanQuery(qry, part); + + GridTestUtils.assertThrows(log, (Callable)() -> { + int remPart = remotePartition(cache.context()).getKey(); + + cache.query(new ScanQuery().setPartition(remPart).setLocal(true)); + + return null; + }, IgniteCheckedException.class, null); + } + finally { + stopAllGrids(); + } + } + + /** + * Scan (with explicit {@code setLocal(true)}, no partition specified) should perform on the local node. + * + * @throws Exception If failed. + */ + public void testScanLocalExplicitNoPart() throws Exception { + cacheMode = CacheMode.PARTITIONED; + backups = 0; + commSpiFactory = new TestLocalCommunicationSpiFactory(); + + try { + Ignite ignite = startGrids(GRID_CNT); + + IgniteCacheProxy cache = fillCache(ignite); + + QueryCursor> qry = + cache.query(new ScanQuery().setLocal(true)); + + assertFalse(qry.getAll().isEmpty()); + } + finally { + stopAllGrids(); + } + } + /** * Scan should perform on the remote node. * From ad858e8ed7f3e4c8ef82e7e553618cac626ff112 Mon Sep 17 00:00:00 2001 From: Toru Yabuki Date: Tue, 4 Sep 2018 14:52:03 +0900 Subject: [PATCH 38/95] IGNITE-9382: Fix Node.js client o process large payloads. - Fixes #4629. Signed-off-by: shroman --- .../nodejs/lib/internal/ClientSocket.js | 35 ++++++++++++++----- .../nodejs/lib/internal/MessageBuffer.js | 6 ++++ 2 files changed, 32 insertions(+), 9 deletions(-) diff --git a/modules/platforms/nodejs/lib/internal/ClientSocket.js b/modules/platforms/nodejs/lib/internal/ClientSocket.js index 1d1a4dd202314..73e11f06120cd 100644 --- a/modules/platforms/nodejs/lib/internal/ClientSocket.js +++ b/modules/platforms/nodejs/lib/internal/ClientSocket.js @@ -110,6 +110,8 @@ class ClientSocket { this._onSocketDisconnect = onSocketDisconnect; this._error = null; this._wasConnected = false; + this._buffer = null; + this._offset = 0; } async connect() { @@ -198,28 +200,43 @@ class ClientSocket { if (this._state === STATE.DISCONNECTED) { return; } - let offset = 0; - while (offset < message.length) { - let buffer = MessageBuffer.from(message, offset); + if (this._buffer) { + this._buffer.concat(message); + this._buffer.position = this._offset; + } + else { + this._buffer = MessageBuffer.from(message, 0); + } + while (this._buffer && this._offset < this._buffer.length) { // Response length - const length = buffer.readInteger(); - offset += length + BinaryUtils.getSize(BinaryUtils.TYPE_CODE.INTEGER); + const length = this._buffer.readInteger() + BinaryUtils.getSize(BinaryUtils.TYPE_CODE.INTEGER); + if (this._buffer.length < this._offset + length) { + break; + } + this._offset += length; + let requestId, isSuccess; const isHandshake = this._state === STATE.HANDSHAKE; if (isHandshake) { // Handshake status - isSuccess = (buffer.readByte() === HANDSHAKE_SUCCESS_STATUS_CODE) + isSuccess = (this._buffer.readByte() === HANDSHAKE_SUCCESS_STATUS_CODE); requestId = this._handshakeRequestId.toString(); } else { // Request id - requestId = buffer.readLong().toString(); + requestId = this._buffer.readLong().toString(); // Status code - isSuccess = (buffer.readInteger() === REQUEST_SUCCESS_STATUS_CODE); + isSuccess = (this._buffer.readInteger() === REQUEST_SUCCESS_STATUS_CODE); } - this._logMessage(requestId, false, buffer.data); + this._logMessage(requestId, false, this._buffer.data); + + const buffer = this._buffer; + if (this._offset === this._buffer.length) { + this._buffer = null; + this._offset = 0; + } if (this._requests.has(requestId)) { const request = this._requests.get(requestId); diff --git a/modules/platforms/nodejs/lib/internal/MessageBuffer.js b/modules/platforms/nodejs/lib/internal/MessageBuffer.js index f1407bfb0c479..dc3e92865455f 100644 --- a/modules/platforms/nodejs/lib/internal/MessageBuffer.js +++ b/modules/platforms/nodejs/lib/internal/MessageBuffer.js @@ -42,6 +42,12 @@ class MessageBuffer { return buf; } + concat(source) { + this._buffer = Buffer.concat([this._buffer, source]); + this._length = this._buffer.length; + this._capacity = this._length; + } + get position() { return this._position; } From 9d46063a978a4c8ca3424af5dc317596f3f5155c Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 4 Sep 2018 11:57:07 +0300 Subject: [PATCH 39/95] IGNITE-9141: SQL: pass error message from mapper to reducer in case of mapping failure. This closes #4536. --- .../messages/GridQueryNextPageResponse.java | 35 +++++- .../h2/twostep/GridReduceQueryExecutor.java | 75 ++++++----- .../query/h2/twostep/GridResultPage.java | 2 - .../query/h2/twostep/ReduceQueryRun.java | 117 +++++++++++++++--- ...pearedCacheWasNotFoundMessageSelfTest.java | 2 +- .../query/h2/twostep/JoinSqlTestHelper.java | 2 + .../h2/twostep/RetryCauseMessageSelfTest.java | 5 + 7 files changed, 181 insertions(+), 57 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java index 1df5c26c5f472..aff90e495244e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java @@ -67,6 +67,9 @@ public class GridQueryNextPageResponse implements Message { /** */ private AffinityTopologyVersion retry; + /** Retry cause description*/ + private String retryCause; + /** Last page flag. */ private boolean last; @@ -235,6 +238,12 @@ public Collection plainRows() { writer.incrementState(); case 9: + if (!writer.writeString("retryCause", retryCause)) + return false; + + writer.incrementState(); + + case 10: if (!writer.writeBoolean("removeMapping", removeMapping)) return false; @@ -323,13 +332,23 @@ public Collection plainRows() { return false; reader.incrementState(); + case 9: + retryCause = reader.readString("retryCause"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: removeMapping = reader.readBoolean("removeMapping"); if (!reader.isLastRead()) return false; reader.incrementState(); + } return reader.afterMessageRead(GridQueryNextPageResponse.class); @@ -342,7 +361,7 @@ public Collection plainRows() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 10; + return 11; } /** @@ -359,6 +378,20 @@ public void retry(AffinityTopologyVersion retry) { this.retry = retry; } + /** + * @return Retry Ccause message. + */ + public String retryCause() { + return retryCause; + } + + /** + * @param retryCause Retry Ccause message. + */ + public void retryCause(String retryCause){ + this.retryCause = retryCause; + } + /** * @return Last page flag. */ 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 910ad1a881bec..586fb5118ef47 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 @@ -65,7 +65,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxSelectForUpdateFuture; import org.apache.ignite.internal.processors.cache.distributed.near.TxTopologyVersionFuture; import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker; -import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; @@ -234,8 +233,7 @@ public void start(final GridKernalContext ctx, final IgniteH2Indexing h2) throws * @param nodeId Left node ID. */ private void handleNodeLeft(ReduceQueryRun r, UUID nodeId) { - // Will attempt to retry. If reduce query was started it will fail on next page fetching. - retry(r, h2.readyTopologyVersion(), nodeId); + r.setStateOnNodeLeave(nodeId, h2.readyTopologyVersion()); } /** @@ -287,12 +285,13 @@ private void onFail(ClusterNode node, GridQueryFailResponse msg) { */ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { if (r != null) { - CacheException e = new CacheException("Failed to execute map query on the node: " + nodeId + ", " + msg); + CacheException e = new CacheException("Failed to execute map query on remote node [nodeId=" + nodeId + + ", errMsg=" + msg + ']'); if (failCode == GridQueryFailResponse.CANCELLED_BY_ORIGINATOR) e.addSuppressed(new QueryCancelledException()); - r.state(e, nodeId); + r.setStateOnException(nodeId, e); } } @@ -300,7 +299,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { * @param node Node. * @param msg Message. */ - private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { + private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse msg) { final long qryReqId = msg.queryRequestId(); final int qry = msg.query(); final int seg = msg.segmentId(); @@ -319,20 +318,13 @@ private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { try { page = new GridResultPage(ctx, node.id(), msg) { @Override public void fetchNextPage() { - Object errState = r.state(); + if (r.hasErrorOrRetry()) { + if (r.exception() != null) + throw r.exception(); - if (errState != null) { - CacheException err0 = errState instanceof CacheException ? (CacheException)errState : null; + assert r.retryCause() != null; - if (err0 != null && err0.getCause() instanceof IgniteClientDisconnectedException) - throw err0; - - CacheException e = new CacheException("Failed to fetch data from node: " + node.id()); - - if (err0 != null) - e.addSuppressed(err0); - - throw e; + throw new CacheException(r.retryCause()); } try { @@ -360,7 +352,7 @@ private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { idx.addPage(page); if (msg.retry() != null) - retry(r, msg.retry(), node.id()); + r.setStateOnRetry(node.id(), msg.retry(), msg.retryCause()); else if (msg.page() == 0) { // Do count down on each first page received. r.latch().countDown(); @@ -372,21 +364,15 @@ else if (msg.page() == 0) { } } - /** - * @param r Query run. - * @param retryVer Retry version. - * @param nodeId Node ID. - */ - private void retry(ReduceQueryRun r, AffinityTopologyVersion retryVer, UUID nodeId) { - r.state(retryVer, nodeId); - } - /** * @param cacheIds Cache IDs. * @return {@code true} If preloading is active. */ private boolean isPreloadingActive(List cacheIds) { for (Integer cacheId : cacheIds) { + if (null == cacheContext(cacheId)) + throw new CacheException(String.format("Cache not found on local node [cacheId=%d]", cacheId)); + if (hasMovingPartitions(cacheContext(cacheId))) return true; } @@ -399,6 +385,8 @@ private boolean isPreloadingActive(List cacheIds) { * @return {@code True} If cache has partitions in {@link GridDhtPartitionState#MOVING} state. */ private boolean hasMovingPartitions(GridCacheContext cctx) { + assert cctx != null; + return !cctx.isLocal() && cctx.topology().hasMovingPartitions(); } @@ -593,9 +581,18 @@ public Iterator> query( final long startTime = U.currentTimeMillis(); + ReduceQueryRun lastRun = null; + for (int attempt = 0;; attempt++) { - if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) - throw new CacheException("Failed to map SQL query to topology."); + if (attempt > 0 && retryTimeout > 0 && (U.currentTimeMillis() - startTime > retryTimeout)) { + UUID retryNodeId = lastRun.retryNodeId(); + String retryCause = lastRun.retryCause(); + + assert !F.isEmpty(retryCause); + + throw new CacheException("Failed to map SQL query to topology on data node [dataNodeId=" + retryNodeId + + ", msg=" + retryCause + ']'); + } if (attempt != 0) { try { @@ -877,29 +874,26 @@ else if (mvccTracker != null) if (send(nodes, req, spec, false)) { awaitAllReplies(r, nodes, cancel); - Object state = r.state(); - - if (state != null) { - if (state instanceof CacheException) { - CacheException err = (CacheException)state; + if (r.hasErrorOrRetry()) { + CacheException err = r.exception(); + if (err != null) { if (err.getCause() instanceof IgniteClientDisconnectedException) throw err; if (wasCancelled(err)) throw new QueryCancelledException(); // Throw correct exception. - throw new CacheException("Failed to run map query remotely." + err.getMessage(), err); + throw new CacheException("Failed to run map query remotely: " + err.getMessage(), err); } - - if (state instanceof AffinityTopologyVersion) { + else { retry = true; // On-the-fly topology change must not be possible in FOR UPDATE case. assert sfuFut == null; // If remote node asks us to retry then we have outdated full partition map. - h2.awaitForReadyTopologyVersion((AffinityTopologyVersion)state); + h2.awaitForReadyTopologyVersion(r.retryTopologyVersion()); } } } @@ -952,6 +946,9 @@ else if (mvccTracker != null) } } else { + assert r != null; + lastRun=r; + if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java index 3c176407f98f0..0cb986b6d9092 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java @@ -72,8 +72,6 @@ public GridResultPage(final GridKernalContext ctx, UUID src, GridQueryNextPageRe Collection plainRows = res.plainRows(); if (plainRows != null) { - assert plainRows instanceof ArrayList; - rowsInPage = plainRows.size(); if (rowsInPage == 0 || ((ArrayList)plainRows).get(0).length == res.columns()) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index df72e8cde583c..7ddd653434daa 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; +import org.apache.ignite.internal.util.typedef.F; import org.h2.jdbc.JdbcConnection; import org.jetbrains.annotations.Nullable; @@ -53,8 +54,8 @@ class ReduceQueryRun { /** */ private final int pageSize; - /** Can be either CacheException in case of error or AffinityTopologyVersion to retry if needed. */ - private final AtomicReference state = new AtomicReference<>(); + /** */ + private final AtomicReference state = new AtomicReference<>(); /** Future controlling {@code SELECT FOR UPDATE} query execution. */ private final GridNearTxSelectForUpdateFuture selectForUpdateFut; @@ -86,30 +87,59 @@ class ReduceQueryRun { } /** - * @param o Fail state object. + * Set state on exception. + * + * @param err error. + * @param nodeId Node ID. + */ + void setStateOnException(@Nullable UUID nodeId, CacheException err) { + setState0(new State(nodeId, err, null, null)); + } + + /** + * Set state on map node leave. + * + * @param nodeId Node ID. + * @param topVer Topology version. + */ + void setStateOnNodeLeave(UUID nodeId, AffinityTopologyVersion topVer) { + setState0(new State(nodeId, null, topVer, "Data node has left the grid during query execution [nodeId=" + + nodeId + ']')); + } + + /** + * Set state on retry due to mapping failure. + * * @param nodeId Node ID. + * @param topVer Topology version. + * @param retryCause Retry cause. */ - void state(Object o, @Nullable UUID nodeId) { - assert o != null; - assert o instanceof CacheException || o instanceof AffinityTopologyVersion : o.getClass(); + void setStateOnRetry(UUID nodeId, AffinityTopologyVersion topVer, String retryCause) { + assert !F.isEmpty(retryCause); - if (!state.compareAndSet(null, o)) + setState0(new State(nodeId, null, topVer, retryCause)); + } + + /** + * + * @param state state + */ + private void setState0(State state){ + if (!this.state.compareAndSet(null, state)) return; while (latch.getCount() != 0) // We don't need to wait for all nodes to reply. latch.countDown(); - CacheException e = o instanceof CacheException ? (CacheException) o : null; - for (GridMergeIndex idx : idxs) // Fail all merge indexes. - idx.fail(nodeId, e); + idx.fail(state.nodeId, state.ex); } /** * @param e Error. */ void disconnected(CacheException e) { - state(e, null); + setStateOnException(null, e); } /** @@ -133,11 +163,45 @@ JdbcConnection connection() { return conn; } + /** */ + boolean hasErrorOrRetry(){ + return state.get() != null; + } + /** - * @return State. + * @return Exception. */ - Object state() { - return state.get(); + CacheException exception() { + State st = state.get(); + + return st != null ? st.ex : null; + } + + /** + * @return Retry topology version. + */ + AffinityTopologyVersion retryTopologyVersion(){ + State st = state.get(); + + return st != null ? st.retryTopVer : null; + } + + /** + * @return Retry bode ID. + */ + UUID retryNodeId() { + State st = state.get(); + + return st != null ? st.nodeId : null; + } + + /** + * @return Retry cause. + */ + String retryCause(){ + State st = state.get(); + + return st != null ? st.retryCause : null; } /** @@ -167,4 +231,29 @@ void latch(CountDownLatch latch) { @Nullable public GridNearTxSelectForUpdateFuture selectForUpdateFuture() { return selectForUpdateFut; } + + /** + * Error state. + */ + private static class State { + /** Affected node (may be null in case of local node failure). */ + private final UUID nodeId; + + /** Error. */ + private final CacheException ex; + + /** Retry topology version. */ + private final AffinityTopologyVersion retryTopVer; + + /** Retry cause. */ + private final String retryCause; + + /** */ + private State(UUID nodeId, CacheException ex, AffinityTopologyVersion retryTopVer, String retryCause){ + this.nodeId = nodeId; + this.ex = ex; + this.retryTopVer = retryTopVer; + this.retryCause = retryCause; + } + } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java index 91af36e72f85c..9928ed6ff2745 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java @@ -62,7 +62,7 @@ public void testDisappearedCacheWasNotFoundMessage() { fail("No CacheException emitted."); } catch (CacheException e) { - assertTrue(e.getMessage(), e.getMessage().contains("Grid cache context is not registered for cache id")); + assertTrue(e.getMessage(), e.getMessage().contains("Cache not found on local node")); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java index 2abab7448b166..fe7821ae29709 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java @@ -26,8 +26,10 @@ public class JoinSqlTestHelper { /** */ private static final int ORG_COUNT = 100; + /** */ private static final int PERSON_PER_ORG_COUNT = 10; + /** */ static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + "where Person.orgId = org.id " + diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java index 1ead3aa5e5837..326988739ec57 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java @@ -56,14 +56,19 @@ public class RetryCauseMessageSelfTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 2; + /** */ private static final String ORG_SQL = "select * from Organization"; + /** */ private static final String ORG = "org"; + /** */ private IgniteCache personCache; + /** */ private IgniteCache orgCache; + /** */ private IgniteH2Indexing h2Idx; From 83e3f123e7ebefaaa47e81bb73d30d813be80cc1 Mon Sep 17 00:00:00 2001 From: Pavel Voronkin Date: Tue, 4 Sep 2018 12:41:14 +0300 Subject: [PATCH 40/95] Refactoring to improve constant usage for file suffixes. - Fixes #4652. Signed-off-by: Dmitriy Govorukhin --- .../GridCacheDatabaseSharedManager.java | 11 ++++----- .../file/FilePageStoreManager.java | 12 +++++++--- .../cache/persistence/wal/FileDescriptor.java | 3 ++- .../wal/FileWriteAheadLogManager.java | 23 +++++++++++------- .../FsyncModeFileWriteAheadLogManager.java | 24 ++++++++++++------- .../IgnitePdsDiskErrorsRecoveringTest.java | 5 ++-- ...IgniteNodeStoppedDuringDisableWALTest.java | 5 ++-- .../persistence/db/wal/WalCompactionTest.java | 5 ++-- .../impl/v2/HadoopV2JobResourceManager.java | 3 ++- .../submitter/command/StartCommand.java | 3 ++- 10 files changed, 57 insertions(+), 37 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index a86b9f5c449f0..f06fd053fb41b 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -217,9 +217,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** Checkpoint file name pattern. */ public static final Pattern CP_FILE_NAME_PATTERN = Pattern.compile("(\\d+)-(.*)-(START|END)\\.bin"); - /** Checkpoint file temporary suffix. This is needed to safe writing checkpoint markers through temporary file and renaming. */ - public static final String FILE_TMP_SUFFIX = ".tmp"; - /** Node started file suffix. */ public static final String NODE_STARTED_FILE_NAME_SUFFIX = "-node-started.bin"; @@ -498,13 +495,13 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu } /** - * Cleanup checkpoint directory from all temporary files {@link #FILE_TMP_SUFFIX}. + * Cleanup checkpoint directory from all temporary files. */ @Override public void cleanupTempCheckpointDirectory() throws IgniteCheckedException { try { try (DirectoryStream files = Files.newDirectoryStream( cpDir.toPath(), - path -> path.endsWith(FILE_TMP_SUFFIX)) + path -> path.endsWith(FilePageStoreManager.TMP_SUFFIX)) ) { for (Path path : files) Files.delete(path); @@ -886,7 +883,7 @@ private void nodeStart(WALPointer ptr) throws IgniteCheckedException { FileWALPointer p = (FileWALPointer)ptr; String fileName = U.currentTimeMillis() + NODE_STARTED_FILE_NAME_SUFFIX; - String tmpFileName = fileName + FILE_TMP_SUFFIX; + String tmpFileName = fileName + FilePageStoreManager.TMP_SUFFIX; ByteBuffer buf = ByteBuffer.allocate(FileWALPointer.POINTER_SIZE); buf.order(ByteOrder.nativeOrder()); @@ -2666,7 +2663,7 @@ private CheckpointEntry prepareCheckpointEntry( */ public void writeCheckpointEntry(ByteBuffer entryBuf, CheckpointEntry cp, CheckpointEntryType type) throws StorageException { String fileName = checkpointFileName(cp, type); - String tmpFileName = fileName + FILE_TMP_SUFFIX; + String tmpFileName = fileName + FilePageStoreManager.TMP_SUFFIX; try { try (FileIO io = ioFactory.create(Paths.get(cpDir.getAbsolutePath(), skipSync ? fileName : tmpFileName).toFile(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java index aa092419a70e6..020f84e82899a 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java @@ -77,6 +77,12 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen /** File suffix. */ public static final String FILE_SUFFIX = ".bin"; + /** Suffix for zip files */ + public static final String ZIP_SUFFIX = ".zip"; + + /** Suffix for tmp files */ + public static final String TMP_SUFFIX = ".tmp"; + /** Partition file prefix. */ public static final String PART_FILE_PREFIX = "part-"; @@ -96,7 +102,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen public static final String CACHE_DATA_FILENAME = "cache_data.dat"; /** */ - public static final String CACHE_DATA_TMP_FILENAME = CACHE_DATA_FILENAME + ".tmp"; + public static final String CACHE_DATA_TMP_FILENAME = CACHE_DATA_FILENAME + TMP_SUFFIX; /** */ public static final String DFLT_STORE_DIR = "db"; @@ -341,7 +347,7 @@ public FilePageStoreManager(GridKernalContext ctx) { if (overwrite || !file.exists() || file.length() == 0) { try { - File tmp = new File(file.getParent(), file.getName() + ".tmp"); + File tmp = new File(file.getParent(), file.getName() + TMP_SUFFIX); tmp.createNewFile(); @@ -596,7 +602,7 @@ private boolean checkAndInitCacheWorkDir(File cacheWorkDir) throws IgniteChecked Path cacheWorkDirPath = cacheWorkDir.toPath(); - Path tmp = cacheWorkDirPath.getParent().resolve(cacheWorkDir.getName() + ".tmp"); + Path tmp = cacheWorkDirPath.getParent().resolve(cacheWorkDir.getName() + TMP_SUFFIX); if (Files.exists(tmp) && Files.isDirectory(tmp) && Files.exists(tmp.resolve(IgniteCacheSnapshotManager.TEMP_FILES_COMPLETENESS_MARKER))) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java index 39052cc92fcd1..a73248a1cab48 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileDescriptor.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.wal; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.util.typedef.internal.SB; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -121,7 +122,7 @@ public String getAbsolutePath() { /** {@inheritDoc} */ @Override public boolean isCompressed() { - return file.getName().endsWith(".zip"); + return file.getName().endsWith(FilePageStoreManager.ZIP_SUFFIX); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java index 61def1e7bd41e..eb5ae95dd31e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java @@ -92,6 +92,7 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; import org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator.AbstractFileDescriptor; @@ -534,7 +535,7 @@ public Collection getAndReserveWalFiles(FileWALPointer low, FileWALPointer String segmentName = FileDescriptor.fileName(i); File file = new File(walArchiveDir, segmentName); - File fileZip = new File(walArchiveDir, segmentName + ".zip"); + File fileZip = new File(walArchiveDir, segmentName + FilePageStoreManager.ZIP_SUFFIX); if (file.exists()) res.add(file); @@ -909,7 +910,7 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { private boolean hasIndex(long absIdx) { String segmentName = FileDescriptor.fileName(absIdx); - String zipSegmentName = FileDescriptor.fileName(absIdx) + ".zip"; + String zipSegmentName = FileDescriptor.fileName(absIdx) + FilePageStoreManager.ZIP_SUFFIX; boolean inArchive = new File(walArchiveDir, segmentName).exists() || new File(walArchiveDir, zipSegmentName).exists(); @@ -1496,7 +1497,7 @@ private void createFile(File file) throws StorageException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); - File tmp = new File(file.getParent(), file.getName() + ".tmp"); + File tmp = new File(file.getParent(), file.getName() + FilePageStoreManager.TMP_SUFFIX); formatFile(tmp); @@ -1911,7 +1912,7 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException String name = FileDescriptor.fileName(absIdx); - File dstTmpFile = new File(walArchiveDir, name + ".tmp"); + File dstTmpFile = new File(walArchiveDir, name + FilePageStoreManager.TMP_SUFFIX); File dstFile = new File(walArchiveDir, name); @@ -2104,9 +2105,10 @@ private void deleteObsoleteRawSegments() { if (currReservedSegment == -1) continue; - File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip" + ".tmp"); + File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + + FilePageStoreManager.ZIP_SUFFIX + FilePageStoreManager.TMP_SUFFIX); - File zip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip"); + File zip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + FilePageStoreManager.ZIP_SUFFIX); File raw = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment)); if (!Files.exists(raw.toPath())) @@ -2265,8 +2267,10 @@ private class FileDecompressor extends GridWorker { if (isCancelled()) break; - File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); - File unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".tmp"); + File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + + FilePageStoreManager.ZIP_SUFFIX); + File unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + + FilePageStoreManager.TMP_SUFFIX); File unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); @@ -3061,7 +3065,8 @@ private RecordsIterator( if (!desc.file().exists()) { FileDescriptor zipFile = new FileDescriptor( - new File(walArchiveDir, FileDescriptor.fileName(desc.idx()) + ".zip")); + new File(walArchiveDir, FileDescriptor.fileName(desc.idx()) + + FilePageStoreManager.ZIP_SUFFIX)); if (!zipFile.file.exists()) { throw new FileNotFoundException("Both compressed and raw segment files are missing in archive " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java index 91d3218572e64..0f15d499b7157 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java @@ -87,6 +87,7 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32; import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; @@ -618,7 +619,7 @@ public Collection getAndReserveWalFiles(FileWALPointer low, FileWALPointer String segmentName = FileDescriptor.fileName(i); File file = new File(walArchiveDir, segmentName); - File fileZip = new File(walArchiveDir, segmentName + ".zip"); + File fileZip = new File(walArchiveDir, segmentName + FilePageStoreManager.ZIP_SUFFIX); if (file.exists()) res.add(file); @@ -815,7 +816,7 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { private boolean hasIndex(long absIdx) { String segmentName = FileDescriptor.fileName(absIdx); - String zipSegmentName = FileDescriptor.fileName(absIdx) + ".zip"; + String zipSegmentName = FileDescriptor.fileName(absIdx) + FilePageStoreManager.ZIP_SUFFIX; boolean inArchive = new File(walArchiveDir, segmentName).exists() || new File(walArchiveDir, zipSegmentName).exists(); @@ -1345,7 +1346,7 @@ private void createFile(File file) throws StorageException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); - File tmp = new File(file.getParent(), file.getName() + ".tmp"); + File tmp = new File(file.getParent(), file.getName() + FilePageStoreManager.TMP_SUFFIX); formatFile(tmp); @@ -1740,7 +1741,7 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedExc String name = FileDescriptor.fileName(absIdx); - File dstTmpFile = new File(walArchiveDir, name + ".tmp"); + File dstTmpFile = new File(walArchiveDir, name + FilePageStoreManager.TMP_SUFFIX); File dstFile = new File(walArchiveDir, name); @@ -1934,9 +1935,11 @@ private void deleteObsoleteRawSegments() { if (currReservedSegment == -1) continue; - File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip" + ".tmp"); + File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + + FilePageStoreManager.ZIP_SUFFIX + FilePageStoreManager.TMP_SUFFIX); - File zip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip"); + File zip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + + FilePageStoreManager.ZIP_SUFFIX); File raw = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment)); if (!Files.exists(raw.toPath())) @@ -2073,8 +2076,10 @@ private class FileDecompressor extends GridWorker { if (isCancelled()) break; - File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); - File unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".tmp"); + File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + + FilePageStoreManager.ZIP_SUFFIX); + File unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + + FilePageStoreManager.TMP_SUFFIX); File unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); @@ -3146,7 +3151,8 @@ private RecordsIterator( if (!desc.file().exists()) { FileDescriptor zipFile = new FileDescriptor( - new File(walArchiveDir, FileDescriptor.fileName(desc.idx()) + ".zip")); + new File(walArchiveDir, FileDescriptor.fileName(desc.idx()) + + FilePageStoreManager.ZIP_SUFFIX)); if (!zipFile.file.exists()) { throw new FileNotFoundException("Both compressed and raw segment files are missing in archive " + diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java index 7efe29b326d20..bd30696a1e260 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; @@ -158,7 +159,7 @@ public void testRecoveringOnCacheInitFail() throws Exception { */ public void testRecoveringOnNodeStartMarkerWriteFail() throws Exception { // Fail to write node start marker tmp file at the second checkpoint. Pass only initial checkpoint. - ioFactory = new FilteringFileIOFactory("started.bin" + GridCacheDatabaseSharedManager.FILE_TMP_SUFFIX, new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 20)); + ioFactory = new FilteringFileIOFactory("started.bin" + FilePageStoreManager.TMP_SUFFIX, new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 20)); IgniteEx grid = startGrid(0); grid.cluster().active(true); @@ -213,7 +214,7 @@ public void testRecoveringOnNodeStartMarkerWriteFail() throws Exception { */ public void testRecoveringOnCheckpointBeginFail() throws Exception { // Fail to write checkpoint start marker tmp file at the second checkpoint. Pass only initial checkpoint. - ioFactory = new FilteringFileIOFactory("START.bin" + GridCacheDatabaseSharedManager.FILE_TMP_SUFFIX, new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 20)); + ioFactory = new FilteringFileIOFactory("START.bin" + FilePageStoreManager.TMP_SUFFIX, new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 20)); final IgniteEx grid = startGrid(0); grid.cluster().active(true); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java index 80198e8af14ee..a744ab10c1671 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java @@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.WalStateManager.WALDisableContext; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; @@ -47,7 +48,7 @@ import static java.nio.file.FileVisitResult.CONTINUE; import static java.nio.file.Files.walkFileTree; import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.CP_FILE_NAME_PATTERN; -import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.FILE_TMP_SUFFIX; + import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.META_STORAGE_NAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX; @@ -208,7 +209,7 @@ private void testStopNodeWithDisableWAL(NodeStopPoint nodeStopPoint) throws Exce boolean failed = false; - if (name.endsWith(FILE_TMP_SUFFIX)) + if (name.endsWith(FilePageStoreManager.TMP_SUFFIX)) failed = true; if (CP_FILE_NAME_PATTERN.matcher(name).matches()) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java index f890d61b2c025..e61745505380b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -163,7 +164,7 @@ private void testApplyingUpdatesFromCompactedWal(boolean switchOffCompressor) th File walDir = new File(dbDir, "wal"); File archiveDir = new File(walDir, "archive"); File nodeArchiveDir = new File(archiveDir, nodeFolderName); - File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0) + ".zip"); + File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0) + FilePageStoreManager.ZIP_SUFFIX); assertTrue(walSegment.exists()); assertTrue(walSegment.length() < WAL_SEGMENT_SIZE / 2); // Should be compressed at least in half. @@ -358,7 +359,7 @@ public void testSeekingStartInCompactedSegment() throws Exception { File walDir = new File(dbDir, "wal"); File archiveDir = new File(walDir, "archive"); File nodeArchiveDir = new File(archiveDir, nodeFolderName); - File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0) + ".zip"); + File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0) + FilePageStoreManager.ZIP_SUFFIX); assertTrue(walSegment.exists()); assertTrue(walSegment.length() < WAL_SEGMENT_SIZE / 2); // Should be compressed at least in half. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java index 52e394be96589..18dbf2097bc13 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java @@ -26,6 +26,7 @@ import org.apache.hadoop.util.RunJar; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils; @@ -234,7 +235,7 @@ private void processFiles(File jobLocDir, @Nullable Object[] files, boolean down if (archiveNameLC.endsWith(".jar")) RunJar.unJar(archiveFile, dstPath); - else if (archiveNameLC.endsWith(".zip")) + else if (archiveNameLC.endsWith(FilePageStoreManager.ZIP_SUFFIX)) FileUtil.unZip(archiveFile, dstPath); else if (archiveNameLC.endsWith(".tar.gz") || archiveNameLC.endsWith(".tgz") || diff --git a/modules/tensorflow/src/main/java/org/apache/ignite/tensorflow/submitter/command/StartCommand.java b/modules/tensorflow/src/main/java/org/apache/ignite/tensorflow/submitter/command/StartCommand.java index 082b3639bb439..0202b8e209c6b 100644 --- a/modules/tensorflow/src/main/java/org/apache/ignite/tensorflow/submitter/command/StartCommand.java +++ b/modules/tensorflow/src/main/java/org/apache/ignite/tensorflow/submitter/command/StartCommand.java @@ -28,6 +28,7 @@ import java.util.zip.ZipOutputStream; import org.apache.commons.io.IOUtils; import org.apache.ignite.Ignite; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.tensorflow.cluster.TensorFlowClusterGatewayManager; import org.apache.ignite.tensorflow.cluster.TensorFlowJobArchive; import picocli.CommandLine; @@ -97,7 +98,7 @@ private byte[] zip(String jobArchivePath) throws IOException { if (file.isDirectory()) return zipDirectory(file); - else if (jobArchivePath.endsWith(".zip")) + else if (jobArchivePath.endsWith(FilePageStoreManager.ZIP_SUFFIX)) return zipArchive(file); else return zipFile(file); From 3f75f9575b9897f686057be523d9443577abe351 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Tue, 4 Sep 2018 13:27:28 +0300 Subject: [PATCH 41/95] IGNITE-9361 Remove IgniteInternalCache.isMongo*Cache() and other remnants. - Fixes #4678. Signed-off-by: Dmitriy Govorukhin --- .../org/apache/ignite/internal/GridTopic.java | 3 --- .../ignite/internal/IgniteNodeAttributes.java | 3 --- .../processors/cache/GridCacheAdapter.java | 18 -------------- .../processors/cache/GridCacheProxyImpl.java | 24 ------------------- .../processors/cache/IgniteInternalCache.java | 14 ----------- .../near/GridNearCacheAdapter.java | 10 -------- modules/core/src/test/config/tests.properties | 5 ---- .../cache/hibernate/HibernateCacheProxy.java | 10 -------- 8 files changed, 87 deletions(-) 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 68cea4f30e0f7..5610b2a2d4d6b 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 @@ -81,9 +81,6 @@ public enum GridTopic { /** */ TOPIC_CONTINUOUS, - /** */ - TOPIC_MONGO, - /** */ TOPIC_TIME_SYNC, 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 61a98a138b6ab..539727d7cd9d7 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 @@ -81,9 +81,6 @@ public final class IgniteNodeAttributes { /** Internal attribute name constant. */ public static final String ATTR_IGFS = ATTR_PREFIX + ".igfs"; - /** Internal attribute name constant. */ - public static final String ATTR_MONGO = ATTR_PREFIX + ".mongo"; - /** Internal attribute name constant. */ public static final String ATTR_DAEMON = ATTR_PREFIX + ".daemon"; 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 159d6810e0521..4481e4d8c35da 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 @@ -276,14 +276,6 @@ public abstract class GridCacheAdapter implements IgniteInternalCache delegate() { } } - /** {@inheritDoc} */ - @Override public boolean isMongoDataCache() { - CacheOperationContext prev = gate.enter(opCtx); - - try { - return delegate.isMongoDataCache(); - } - finally { - gate.leave(prev); - } - } - - /** {@inheritDoc} */ - @Override public boolean isMongoMetaCache() { - CacheOperationContext prev = gate.enter(opCtx); - - try { - return delegate.isMongoMetaCache(); - } - finally { - gate.leave(prev); - } - } - /** {@inheritDoc} */ @Override public Map getAll(@Nullable Collection keys) throws IgniteCheckedException { CacheOperationContext prev = gate.enter(opCtx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java index bbedef81ea137..cba2228f0323e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java @@ -1657,20 +1657,6 @@ public IgniteInternalFuture removeAllConflictAsync( */ public long igfsDataSpaceUsed(); - /** - * Checks whether this cache is Mongo data cache. - * - * @return {@code True} if this cache is mongo data cache. - */ - public boolean isMongoDataCache(); - - /** - * Checks whether this cache is Mongo meta cache. - * - * @return {@code True} if this cache is mongo meta cache. - */ - public boolean isMongoMetaCache(); - /** * @param keepBinary Keep binary flag. * @param p Optional key/value predicate. 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 672693c4bb1b2..39047813e85e5 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 @@ -395,16 +395,6 @@ public Set> nearEntries() { dht().onIgfsDataSizeChanged(delta); } - /** {@inheritDoc} */ - @Override public boolean isMongoDataCache() { - return dht().isMongoDataCache(); - } - - /** {@inheritDoc} */ - @Override public boolean isMongoMetaCache() { - return dht().isMongoMetaCache(); - } - /** {@inheritDoc} */ @Override public List> splitClearLocally(boolean srv, boolean near, boolean readers) { diff --git a/modules/core/src/test/config/tests.properties b/modules/core/src/test/config/tests.properties index 718d66107f550..b39e3ca318a25 100644 --- a/modules/core/src/test/config/tests.properties +++ b/modules/core/src/test/config/tests.properties @@ -146,8 +146,3 @@ ssl.keystore.trustboth.path=@{IGNITE_HOME}/modules/clients/src/test/keystore/ca/ # Hadoop home directory. hadoop.home=@{HADOOP_HOME} - -# Sharded mongo properties -mongos.host=192.168.2.10 -mongos.port=27017 -sharded.db.name=gg-test-db diff --git a/modules/hibernate-core/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java b/modules/hibernate-core/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java index 70e55de3a572e..fdb87f0c20201 100644 --- a/modules/hibernate-core/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java +++ b/modules/hibernate-core/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java @@ -601,16 +601,6 @@ public HibernateKeyTransformer keyTransformer(){ return delegate.igfsDataSpaceUsed(); } - /** {@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(); From ca86307e7e8dd30900fffff07109253013c4ec17 Mon Sep 17 00:00:00 2001 From: NSAmelchev Date: Tue, 4 Sep 2018 15:08:38 +0300 Subject: [PATCH 42/95] IGNITE-8829 Annotated configuration properties on TcpCommunicationSpi that lack required annotations - Fixes #4583. Signed-off-by: Alexey Goncharuk --- .../tcp/TcpCommunicationSpi.java | 71 ++++++++++++------- 1 file changed, 45 insertions(+), 26 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 77e85c13ecbec..4ab1dd42c2ab0 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -204,16 +204,22 @@ *

    Optional

    * The following configuration parameters are optional: *
      + *
    • Address resolver (see {@link #setAddressResolver(AddressResolver)}
    • *
    • Node local IP address (see {@link #setLocalAddress(String)})
    • *
    • Node local port number (see {@link #setLocalPort(int)})
    • *
    • Local port range (see {@link #setLocalPortRange(int)}
    • + *
    • Use paired connections (see {@link #setUsePairedConnections(boolean)}
    • *
    • Connections per node (see {@link #setConnectionsPerNode(int)})
    • + *
    • Shared memory port (see {@link #setSharedMemoryPort(int)}
    • *
    • Idle connection timeout (see {@link #setIdleConnectionTimeout(long)})
    • *
    • Direct or heap buffer allocation (see {@link #setDirectBuffer(boolean)})
    • *
    • Direct or heap buffer allocation for sending (see {@link #setDirectSendBuffer(boolean)})
    • *
    • Count of selectors and selector threads for NIO server (see {@link #setSelectorsCount(int)})
    • + *
    • Selector thread busy-loop iterations (see {@link #setSelectorSpins(long)}
    • *
    • {@code TCP_NODELAY} socket option for sockets (see {@link #setTcpNoDelay(boolean)})
    • + *
    • Filter reachable addresses (see {@link #setFilterReachableAddresses(boolean)}
    • *
    • Message queue limit (see {@link #setMessageQueueLimit(int)})
    • + *
    • Slow client queue limit (see {@link #setSlowClientQueueLimit(int)})
    • *
    • Connect timeout (see {@link #setConnectTimeout(long)})
    • *
    • Maximum connect timeout (see {@link #setMaxConnectTimeout(long)})
    • *
    • Reconnect attempts count (see {@link #setReconnectCount(int)})
    • @@ -382,7 +388,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati private final GridNioServerListener srvLsnr = new GridNioServerListenerAdapter() { @Override public void onSessionWriteTimeout(GridNioSession ses) { - LT.warn(log,"Communication SPI session write timed out (consider increasing " + + LT.warn(log, "Communication SPI session write timed out (consider increasing " + "'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() + ", writeTimeout=" + sockWriteTimeout + ']'); @@ -499,7 +505,7 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { boolean unknownNode = true; if (discoverySpi instanceof TcpDiscoverySpi) { - TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi; + TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi)discoverySpi; ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId); @@ -511,7 +517,7 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { } } else if (discoverySpi instanceof IgniteDiscoverySpi) - unknownNode = !((IgniteDiscoverySpi) discoverySpi).knownNode(sndId); + unknownNode = !((IgniteDiscoverySpi)discoverySpi).knownNode(sndId); if (unknownNode) { U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + ", ses=" + ses + ']'); @@ -636,7 +642,7 @@ else if (discoverySpi instanceof IgniteDiscoverySpi) if (log.isDebugEnabled()) log.debug("Received incoming connection from remote node " + - "[rmtNode=" + rmtNode.id() + ", reserved=" + reserved + + "[rmtNode=" + rmtNode.id() + ", reserved=" + reserved + ", recovery=" + recoveryDesc + ']'); if (reserved) { @@ -655,9 +661,9 @@ else if (discoverySpi instanceof IgniteDiscoverySpi) if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) { if (log.isInfoEnabled()) { log.info("Received incoming connection from remote node while " + - "connecting to this node, rejecting [locNode=" + locNode.id() + - ", locNodeOrder=" + locNode.order() + ", rmtNode=" + rmtNode.id() + - ", rmtNodeOrder=" + rmtNode.order() + ']'); + "connecting to this node, rejecting [locNode=" + locNode.id() + + ", locNodeOrder=" + locNode.order() + ", rmtNode=" + rmtNode.id() + + ", rmtNodeOrder=" + rmtNode.order() + ']'); } ses.send(new RecoveryLastReceivedMessage(ALREADY_CONNECTED)); @@ -928,7 +934,7 @@ class ConnectClosureNew implements IgniteInClosure { catch (IgniteCheckedException e) { if (log.isDebugEnabled()) log.debug("Failed to send recovery handshake " + - "[rmtNode=" + rmtNode.id() + ", err=" + e + ']'); + "[rmtNode=" + rmtNode.id() + ", err=" + e + ']'); recoveryDesc.release(); } @@ -1010,14 +1016,14 @@ class ConnectClosure implements IgniteInClosure { msgFut.get(); GridTcpNioCommunicationClient client = - connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient); + connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient); fut.onDone(client); } catch (IgniteCheckedException e) { if (log.isDebugEnabled()) log.debug("Failed to send recovery handshake " + - "[rmtNode=" + rmtNode.id() + ", err=" + e + ']'); + "[rmtNode=" + rmtNode.id() + ", err=" + e + ']'); recoveryDesc.release(); @@ -1209,6 +1215,15 @@ public TcpCommunicationSpi setAddressResolver(AddressResolver addrRslvr) { return this; } + /** + * See {@link #setAddressResolver(AddressResolver)}. + * + * @return Address resolver. + */ + public AddressResolver getAddressResolver() { + return addrRslvr; + } + /** * Injects resources. * @@ -1330,9 +1345,10 @@ public boolean isUsePairedConnections() { * Default is {@code false}. * * @param usePairedConnections {@code true} to use paired connections and {@code false} otherwise. - * @see #getConnectionsPerNode() * @return {@code this} for chaining. + * @see #getConnectionsPerNode() */ + @IgniteSpiConfiguration(optional = true) public TcpCommunicationSpi setUsePairedConnections(boolean usePairedConnections) { this.usePairedConnections = usePairedConnections; @@ -1345,9 +1361,10 @@ public TcpCommunicationSpi setUsePairedConnections(boolean usePairedConnections) * half for outgoing messages. * * @param maxConnectionsPerNode Number of connections per node. - * @see #isUsePairedConnections() * @return {@code this} for chaining. + * @see #isUsePairedConnections() */ + @IgniteSpiConfiguration(optional = true) public TcpCommunicationSpi setConnectionsPerNode(int maxConnectionsPerNode) { this.connectionsPerNode = maxConnectionsPerNode; @@ -1357,7 +1374,7 @@ public TcpCommunicationSpi setConnectionsPerNode(int maxConnectionsPerNode) { /** * See {@link #setConnectionsPerNode(int)}. * - * @return Number of connections per node. + * @return Number of connections per node. */ public int getConnectionsPerNode() { return connectionsPerNode; @@ -1513,7 +1530,8 @@ public TcpCommunicationSpi setConnectTimeout(long connTimeout) { * See {@link #setConnectTimeout(long)}. * * @return Connect timeout. - */public long getConnectTimeout() { + */ + public long getConnectTimeout() { return connTimeout; } @@ -1671,6 +1689,7 @@ public long getSelectorSpins() { * @param selectorSpins Selector thread busy-loop iterations. * @return {@code this} for chaining. */ + @IgniteSpiConfiguration(optional = true) public TcpCommunicationSpi setSelectorSpins(long selectorSpins) { this.selectorSpins = selectorSpins; @@ -1832,6 +1851,7 @@ public int getSlowClientQueueLimit() { * @param slowClientQueueLimit Slow client queue limit. * @return {@code this} for chaining. */ + @IgniteSpiConfiguration(optional = true) public TcpCommunicationSpi setSlowClientQueueLimit(int slowClientQueueLimit) { this.slowClientQueueLimit = slowClientQueueLimit; @@ -2223,7 +2243,7 @@ public int boundPort() { log.debug(startInfo()); } - /** {@inheritDoc} }*/ + /** {@inheritDoc} } */ @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException { spiCtx.registerPort(boundTcpPort, IgnitePortProtocol.TCP); @@ -2331,12 +2351,12 @@ private GridNioServer resetNioServer() throws IgniteCheckedException { IgniteBiInClosure queueSizeMonitor = !clientMode && slowClientQueueLimit > 0 ? - new CI2() { - @Override public void apply(GridNioSession ses, Integer qSize) { - checkClientQueueSize(ses, qSize); - } - } : - null; + new CI2() { + @Override public void apply(GridNioSession ses, Integer qSize) { + checkClientQueueSize(ses, qSize); + } + } : + null; GridNioFilter[] filters; @@ -3262,7 +3282,7 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) GridNioSession ses = recoveryDesc.session(); if (ses != null) { - while(ses.closeTime() == 0) + while (ses.closeTime() == 0) ses.close(); } @@ -3485,7 +3505,6 @@ private boolean isLocalNodeAddress(InetSocketAddress addr) { * @param node Remote node. * @param addrs Remote node addresses. * @param errs TCP client creation errors. - * * @throws IgniteCheckedException If failed. */ protected void processClientCreationError( @@ -3592,8 +3611,8 @@ private void safeShmemHandshake( * @param timeout Timeout for handshake. * @param sslMeta Session meta. * @param handshakeConnIdx Non null connection index if need send it in handshake. - * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout. * @return Handshake response. + * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout. */ @SuppressWarnings("ThrowFromFinallyBlock") private long safeTcpHandshake( @@ -4113,7 +4132,7 @@ private class DiscoveryListener implements GridLocalEventListener, HighPriorityL /** {@inheritDoc} */ @Override public void onEvent(Event evt) { assert evt instanceof DiscoveryEvent : evt; - assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED ; + assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED; onNodeLeft(((DiscoveryEvent)evt).eventNode().id()); } @@ -4194,7 +4213,7 @@ private ShmemWorker(IpcEndpoint endpoint) { srvLsnr, writerFactory, new GridNioCodecFilter( - new GridDirectParser(log.getLogger(GridDirectParser.class),msgFactory, readerFactory), + new GridDirectParser(log.getLogger(GridDirectParser.class), msgFactory, readerFactory), log, true), new GridConnectionBytesVerifyFilter(log) From 57ba2f157bf1b1e8bf5518722ae89cbdc2276132 Mon Sep 17 00:00:00 2001 From: vd-pyatkov Date: Tue, 4 Sep 2018 15:55:41 +0300 Subject: [PATCH 43/95] IGNITE-9448 Updated ZooKeeper version to 3.4.13 - Fixes #4671. Signed-off-by: Alexey Goncharuk --- .../ipfinder/zk/ZookeeperIpFinderTest.java | 3 +- .../curator/FixedTestingQuorumPeerMain.java | 87 +++++++ .../ipfinder/zk/curator/TestingCluster.java | 237 ++++++++++++++++++ .../zk/curator/TestingZooKeeperServer.java | 169 +++++++++++++ ...ookeeperDiscoverySpiAbstractTestSuite.java | 2 +- .../zk/ZookeeperDiscoverySpiTestSuite2.java | 2 +- .../zk/internal/ZookeeperClientTest.java | 2 +- .../internal/ZookeeperDiscoverySpiTest.java | 4 +- parent/pom.xml | 2 +- 9 files changed, 501 insertions(+), 7 deletions(-) create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/FixedTestingQuorumPeerMain.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingCluster.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingZooKeeperServer.java diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java index f85c6a3c9b7d1..ec5630f8a3209 100644 --- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java @@ -26,7 +26,6 @@ import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.retry.RetryNTimes; import org.apache.curator.test.InstanceSpec; -import org.apache.curator.test.TestingCluster; import org.apache.curator.utils.CloseableUtils; import org.apache.ignite.Ignite; import org.apache.ignite.configuration.IgniteConfiguration; @@ -34,6 +33,7 @@ import org.apache.ignite.events.EventType; import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.zk.curator.TestingCluster; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Rule; @@ -85,6 +85,7 @@ public ZookeeperIpFinderTest() { // start the ZK cluster zkCluster = new TestingCluster(ZK_CLUSTER_SIZE); + zkCluster.start(); // start the Curator client so we can perform assertions on the ZK state later diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/FixedTestingQuorumPeerMain.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/FixedTestingQuorumPeerMain.java new file mode 100644 index 0000000000000..e0a416e8e9b96 --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/FixedTestingQuorumPeerMain.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.tcp.ipfinder.zk.curator; + +import org.apache.curator.test.ZooKeeperMainFace; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; +import org.apache.zookeeper.server.quorum.QuorumPeerMain; +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.channels.ServerSocketChannel; + +/** + */ +public class FixedTestingQuorumPeerMain extends QuorumPeerMain implements ZooKeeperMainFace { + @Override public void runFromConfig(QuorumPeerConfig config) throws IOException { + quorumPeer = QuorumPeer.testingQuorumPeer(); + super.runFromConfig(config); + } + + /** {@inheritDoc} */ + @Override public void kill() { + try { + if (quorumPeer != null) { + Field cnxnFactoryField = QuorumPeer.class.getDeclaredField("cnxnFactory"); + + cnxnFactoryField.setAccessible(true); + + ServerCnxnFactory cnxnFactory = (ServerCnxnFactory)cnxnFactoryField.get(quorumPeer); + + cnxnFactory.closeAll(); + + Field ssField = cnxnFactory.getClass().getDeclaredField("ss"); + + ssField.setAccessible(true); + + ServerSocketChannel ss = (ServerSocketChannel)ssField.get(cnxnFactory); + + ss.close(); + } + close(); + } + catch (Exception e) { + e.printStackTrace(); + } + } + + /** {@inheritDoc} */ + @Override public QuorumPeer getQuorumPeer() { + return quorumPeer; + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + if (quorumPeer != null) + quorumPeer.shutdown(); + } + + /** {@inheritDoc} */ + @Override public void blockUntilStarted() throws Exception { + while (quorumPeer == null) { + try { + Thread.sleep(100); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingCluster.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingCluster.java new file mode 100644 index 0000000000000..c6e8d7268cbaf --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingCluster.java @@ -0,0 +1,237 @@ +/* + * 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.zk.curator; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import org.apache.curator.test.ByteCodeRewrite; +import org.apache.curator.test.InstanceSpec; +import org.apache.curator.test.QuorumConfigBuilder; +import org.apache.zookeeper.ZooKeeper; +import java.io.Closeable; +import java.io.IOException; +import java.lang.reflect.Method; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +/** + * Manages an internally running ensemble of ZooKeeper servers. FOR TESTING PURPOSES ONLY. + * This class is a copy of {{org.apache.curator.test.TestingCluster}}, + * but have very small change, that allow to run testing cluster with ZooKeeper 2.4.13 ver. + */ +public class TestingCluster implements Closeable { + static { + ByteCodeRewrite.apply(); + } + + /** Servers. */ + private final List servers; + + /** + * Creates an ensemble comprised of n servers. Each server will use + * a temp directory and random ports + * + * @param instanceQty number of servers to create in the ensemble + */ + public TestingCluster(int instanceQty) { + this(makeSpecs(instanceQty)); + } + + /** + * Creates an ensemble using the given server specs + * + * @param specs the server specs + */ + public TestingCluster(InstanceSpec... specs) { + this(listToMap(ImmutableList.copyOf(specs))); + } + + /** + * Creates an ensemble using the given server specs + * + * @param specs the server specs + */ + public TestingCluster(Collection specs) { + this(listToMap(specs)); + } + + /** + * Creates an ensemble using the given server specs + * + * @param specs map of an instance spec to its set of quorum instances. Allows simulation of an ensemble with + * instances having different config peers + */ + public TestingCluster(Map> specs) { + ImmutableList.Builder serverBuilder = ImmutableList.builder(); + for (Map.Entry> entry : specs.entrySet()) { + List instanceSpecs = Lists.newArrayList(entry.getValue()); + int index = instanceSpecs.indexOf(entry.getKey()); + Preconditions.checkState(index >= 0, entry.getKey() + " not found in specs"); + QuorumConfigBuilder builder = new QuorumConfigBuilder(instanceSpecs); + serverBuilder.add(new TestingZooKeeperServer(builder, index)); + } + servers = serverBuilder.build(); + } + + /** + * Returns the set of servers in the ensemble + * + * @return set of servers + */ + public Collection getInstances() { + Iterable transformed = Iterables.transform + ( + servers, + new Function() { + @Override + public InstanceSpec apply(TestingZooKeeperServer server) { + return server.getInstanceSpec(); + } + } + ); + return Lists.newArrayList(transformed); + } + + public List getServers() { + return Lists.newArrayList(servers); + } + + /** + * Returns the connection string to pass to the ZooKeeper constructor + * + * @return connection string + */ + public String getConnectString() { + StringBuilder str = new StringBuilder(); + for (InstanceSpec spec : getInstances()) { + if (str.length() > 0) + str.append(","); + + str.append(spec.getConnectString()); + } + return str.toString(); + } + + /** + * Start the ensemble. The cluster must be started before use. + * + * @throws Exception errors + */ + public void start() throws Exception { + for (TestingZooKeeperServer server : servers) + server.start(); + + } + + /** + * Shutdown the ensemble WITHOUT freeing resources, etc. + */ + public void stop() throws IOException { + for (TestingZooKeeperServer server : servers) + server.stop(); + + } + + /** + * Shutdown the ensemble, free resources, etc. If temp directories were used, they + * are deleted. You should call this in a finally block. + * + * @throws IOException errors + */ + @Override public void close() throws IOException { + for (TestingZooKeeperServer server : servers) + server.close(); + } + + /** + * Kills the given server. This simulates the server unexpectedly crashing + * + * @param instance server to kill + * @return true if the instance was found + * @throws Exception errors + */ + public boolean killServer(InstanceSpec instance) throws Exception { + for (TestingZooKeeperServer server : servers) { + if (server.getInstanceSpec().equals(instance)) { + server.kill(); + return true; + } + } + return false; + } + + /** + * Restart the given server of the cluster + * + * @param instance server instance + * @return true of the server was found + * @throws Exception errors + */ + public boolean restartServer(InstanceSpec instance) throws Exception { + for (TestingZooKeeperServer server : servers) { + if (server.getInstanceSpec().equals(instance)) { + server.restart(); + return true; + } + } + return false; + } + + /** + * Given a ZooKeeper instance, returns which server it is connected to + * + * @param client ZK instance + * @return the server + * @throws Exception errors + */ + public InstanceSpec findConnectionInstance(ZooKeeper client) throws Exception { + Method m = client.getClass().getDeclaredMethod("testableRemoteSocketAddress"); + m.setAccessible(true); + InetSocketAddress address = (InetSocketAddress)m.invoke(client); + if (address != null) { + for (TestingZooKeeperServer server : servers) { + if (server.getInstanceSpec().getPort() == address.getPort()) + return server.getInstanceSpec(); + } + } + + return null; + } + + private static Map> makeSpecs(int instanceQty) { + ImmutableList.Builder builder = ImmutableList.builder(); + for (int i = 0; i < instanceQty; ++i) + builder.add(InstanceSpec.newInstanceSpec()); + + return listToMap(builder.build()); + } + + private static Map> listToMap(Collection list) { + ImmutableMap.Builder> mapBuilder = ImmutableMap.builder(); + for (InstanceSpec spec : list) + mapBuilder.put(spec, list); + + return mapBuilder.build(); + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingZooKeeperServer.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingZooKeeperServer.java new file mode 100644 index 0000000000000..808e617e17edb --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/curator/TestingZooKeeperServer.java @@ -0,0 +1,169 @@ +/* + * 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.zk.curator; + +import org.apache.curator.test.DirectoryUtils; +import org.apache.curator.test.InstanceSpec; +import org.apache.curator.test.QuorumConfigBuilder; +import org.apache.curator.test.TestingZooKeeperMain; +import org.apache.curator.test.ZooKeeperMainFace; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; +import org.apache.zookeeper.server.quorum.QuorumPeerMain; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.Closeable; +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class TestingZooKeeperServer extends QuorumPeerMain implements Closeable { + /** Logger. */ + private static final Logger logger = LoggerFactory.getLogger(org.apache.curator.test.TestingZooKeeperServer.class); + + /** Config builder. */ + private final QuorumConfigBuilder configBuilder; + /** This instance index. */ + private final int thisInstanceIndex; + /** Main. */ + private volatile ZooKeeperMainFace main; + /** State. */ + private final AtomicReference state = new AtomicReference<>(State.LATENT); + + /** + * Server state. + */ + private enum State { + /** Latent. */ + LATENT, + /** Started. */ + STARTED, + /** Stopped. */ + STOPPED, + /** Closed. */ + CLOSED + } + + /** + * @param configBuilder Config builder. + */ + public TestingZooKeeperServer(QuorumConfigBuilder configBuilder) { + this(configBuilder, 0); + } + + /** + * @param configBuilder Config builder. + * @param thisInstanceIndex This instance index. + */ + public TestingZooKeeperServer(QuorumConfigBuilder configBuilder, int thisInstanceIndex) { + this.configBuilder = configBuilder; + this.thisInstanceIndex = thisInstanceIndex; + main = (configBuilder.size() > 1) ? new FixedTestingQuorumPeerMain() : new TestingZooKeeperMain(); + } + + /** {@inheritDoc} */ + public QuorumPeer getQuorumPeer() { + return main.getQuorumPeer(); + } + + /** + * + */ + public Collection getInstanceSpecs() { + return configBuilder.getInstanceSpecs(); + } + + public void kill() { + main.kill(); + state.set(State.STOPPED); + } + + /** + * Restart the server. If the server is running it will be stopped and then + * started again. If it is not running (in a LATENT or STOPPED state) then + * it will be restarted. If it is in a CLOSED state then an exception will + * be thrown. + * + * @throws Exception + */ + public void restart() throws Exception { + // Can't restart from a closed state as all the temporary data is gone + if (state.get() == State.CLOSED) + throw new IllegalStateException("Cannot restart a closed instance"); + + // If the server's currently running then stop it. + if (state.get() == State.STARTED) + stop(); + + // Set to a LATENT state so we can restart + state.set(State.LATENT); + + main = (configBuilder.size() > 1) ? new FixedTestingQuorumPeerMain() : new TestingZooKeeperMain(); + start(); + } + + /** + * + */ + public void stop() throws IOException { + if (state.compareAndSet(State.STARTED, State.STOPPED)) + main.close(); + } + + /** + * + */ + public InstanceSpec getInstanceSpec() { + return configBuilder.getInstanceSpec(thisInstanceIndex); + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + stop(); + + if (state.compareAndSet(State.STOPPED, State.CLOSED)) { + InstanceSpec spec = getInstanceSpec(); + if (spec.deleteDataDirectoryOnClose()) + DirectoryUtils.deleteRecursively(spec.getDataDirectory()); + } + } + + /** + * + */ + public void start() throws Exception { + if (!state.compareAndSet(State.LATENT, State.STARTED)) + return; + + new Thread(new Runnable() { + public void run() { + try { + QuorumPeerConfig config = configBuilder.buildConfig(thisInstanceIndex); + main.runFromConfig(config); + } + catch (Exception e) { + logger.error(String.format("From testing server (random state: %s) for instance: %s", String.valueOf(configBuilder.isFromRandom()), getInstanceSpec()), e); + } + } + }).start(); + + main.blockUntilStarted(); + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java index c5d34884a8c08..9f0cc6d0f7439 100644 --- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java @@ -22,7 +22,7 @@ import java.util.List; import junit.framework.TestSuite; import org.apache.curator.test.InstanceSpec; -import org.apache.curator.test.TestingCluster; +import org.apache.ignite.spi.discovery.tcp.ipfinder.zk.curator.TestingCluster; import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.spi.discovery.DiscoverySpi; diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java index 012366f70abe6..3b8ddeefefe1a 100644 --- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java @@ -18,7 +18,7 @@ package org.apache.ignite.spi.discovery.zk; import junit.framework.TestSuite; -import org.apache.curator.test.TestingCluster; +import org.apache.ignite.spi.discovery.tcp.ipfinder.zk.curator.TestingCluster; import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest; import org.apache.ignite.internal.IgniteClientReconnectCacheTest; import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDataStructuresTest; diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java index 7c9ec51b094b6..e3b91c74889f7 100644 --- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java @@ -26,7 +26,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.curator.test.TestingCluster; +import org.apache.ignite.spi.discovery.tcp.ipfinder.zk.curator.TestingCluster; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java index 19ca5f23fcf46..f249b59d0372a 100644 --- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java @@ -52,8 +52,8 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryNTimes; -import org.apache.curator.test.TestingCluster; -import org.apache.curator.test.TestingZooKeeperServer; +import org.apache.ignite.spi.discovery.tcp.ipfinder.zk.curator.TestingCluster; +import org.apache.ignite.spi.discovery.tcp.ipfinder.zk.curator.TestingZooKeeperServer; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; diff --git a/parent/pom.xml b/parent/pom.xml index 76e39b79e29fa..f5b49ba39a8a4 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -122,7 +122,7 @@ 2.2.0 0.8.3 0.5 - 3.4.6 + 3.4.13 * From 5dcffcdded12654a6d252ef777f05818c1a7c7e5 Mon Sep 17 00:00:00 2001 From: Roman Guseinov Date: Tue, 4 Sep 2018 16:20:16 +0300 Subject: [PATCH 44/95] IGNITE-8987 Fixed Ignite hang on getting of atomic structures after autoactivation - Fixes #4664. Signed-off-by: Alexey Goncharuk --- .../DataStructuresProcessor.java | 5 ++- ...nitePersistentStoreDataStructuresTest.java | 45 ++++++++++++++++++- 2 files changed, 48 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java index 8f6876c1a9ad7..2c77f17c40307 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java @@ -193,7 +193,10 @@ public DataStructuresProcessor(GridKernalContext ctx) { * */ public void onBeforeActivate() { - initLatch = new CountDownLatch(1); + CountDownLatch latch0 = initLatch; + + if (latch0 == null || latch0.getCount() == 0) + initLatch = new CountDownLatch(1); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java index 53e014efaf12e..855b035563aa5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreDataStructuresTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence; +import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteAtomicLong; import org.apache.ignite.IgniteAtomicSequence; @@ -30,6 +31,8 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; 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; @@ -43,6 +46,9 @@ public class IgnitePersistentStoreDataStructuresTest extends GridCommonAbstractT /** */ private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + /** */ + private static volatile boolean autoActivationEnabled = false; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); @@ -56,7 +62,7 @@ public class IgnitePersistentStoreDataStructuresTest extends GridCommonAbstractT cfg.setDataStorageConfiguration(memCfg); - cfg.setAutoActivationEnabled(false); + cfg.setAutoActivationEnabled(autoActivationEnabled); return cfg; } @@ -71,6 +77,8 @@ public class IgnitePersistentStoreDataStructuresTest extends GridCommonAbstractT super.beforeTest(); cleanPersistenceDir(); + + autoActivationEnabled = false; } /** {@inheritDoc} */ @@ -159,6 +167,41 @@ public void testSequence() throws Exception { assertTrue(sequence.incrementAndGet() > i); } + /** + * @throws Exception If failed. + */ + public void testSequenceAfterAutoactivation() throws Exception { + final String seqName = "testSequence"; + + autoActivationEnabled = true; + + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + ignite.atomicSequence(seqName, 0, true); + + stopAllGrids(true); + + final Ignite node = startGrids(2); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + // Should not hang. + node.atomicSequence(seqName, 0, false); + } + }); + + try { + fut.get(10, TimeUnit.SECONDS); + } + catch (IgniteFutureTimeoutCheckedException e) { + fut.cancel(); + + fail("Ignite was stuck on getting the atomic sequence after autoactivation."); + } + } + /** * @throws Exception If failed. */ From e9c54c96634743989141584acf952bf256131a9f Mon Sep 17 00:00:00 2001 From: Alexey Platonov Date: Tue, 4 Sep 2018 18:11:48 +0300 Subject: [PATCH 45/95] IGNITE-9387: [ML] Model updating this closes #4659 --- ...DBOnTreesClassificationTrainerExample.java | 3 +- .../GDBOnTreesRegressionTrainerExample.java | 3 +- .../ml/clustering/kmeans/KMeansTrainer.java | 27 ++- .../ml/composition/BaggingModelTrainer.java | 20 +++ .../ml/composition/ModelsComposition.java | 10 +- .../composition/ModelsCompositionFormat.java | 61 +++++++ .../ml/composition/boosting/GDBTrainer.java | 19 +- .../ignite/ml/knn/NNClassificationModel.java | 16 ++ .../ml/knn/ann/ANNClassificationModel.java | 15 +- .../ml/knn/ann/ANNClassificationTrainer.java | 93 +++++++--- .../ignite/ml/knn/ann/ANNModelFormat.java | 12 +- .../KNNClassificationModel.java | 40 ++++- .../KNNClassificationTrainer.java | 20 ++- .../knn/regression/KNNRegressionTrainer.java | 19 +- .../ml/math/isolve/lsqr/AbstractLSQR.java | 6 +- .../ml/math/isolve/lsqr/LSQROnHeap.java | 2 +- .../org/apache/ignite/ml/nn/MLPTrainer.java | 25 ++- .../preprocessing/PreprocessingTrainer.java | 3 +- .../linear/LinearRegressionLSQRTrainer.java | 27 ++- .../linear/LinearRegressionSGDTrainer.java | 97 +++++++++- .../LogisticRegressionSGDTrainer.java | 42 ++++- .../LogRegressionMultiClassModel.java | 9 + .../LogRegressionMultiClassTrainer.java | 33 +++- .../SVMLinearBinaryClassificationTrainer.java | 64 +++++-- ...VMLinearMultiClassClassificationModel.java | 9 + ...LinearMultiClassClassificationTrainer.java | 82 +++++++-- .../ignite/ml/trainers/DatasetTrainer.java | 166 ++++++++++++++++++ .../apache/ignite/ml/tree/DecisionTree.java | 23 +++ .../RandomForestClassifierTrainer.java | 8 +- .../randomforest/RandomForestTrainer.java | 28 ++- .../ml/clustering/KMeansTrainerTest.java | 82 ++++++--- .../ignite/ml/common/CollectionsTest.java | 9 +- .../ignite/ml/common/LocalModelsTest.java | 5 +- .../composition/boosting/GDBTrainerTest.java | 2 +- .../ignite/ml/knn/ANNClassificationTest.java | 44 +++++ .../ignite/ml/knn/KNNClassificationTest.java | 39 ++++ .../ignite/ml/knn/KNNRegressionTest.java | 40 +++++ .../apache/ignite/ml/nn/MLPTrainerTest.java | 64 +++++++ .../LinearRegressionLSQRTrainerTest.java | 51 ++++++ .../LinearRegressionSGDTrainerTest.java | 62 +++++++ .../logistic/LogRegMultiClassTrainerTest.java | 58 ++++++ .../LogisticRegressionSGDTrainerTest.java | 46 +++++ .../ignite/ml/svm/SVMBinaryTrainerTest.java | 41 +++++ .../ml/svm/SVMMultiClassTrainerTest.java | 43 +++++ .../RandomForestClassifierTrainerTest.java | 33 +++- .../RandomForestRegressionTrainerTest.java | 31 ++++ 46 files changed, 1503 insertions(+), 129 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsCompositionFormat.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java index 130b91a76b7af..075eab2ea8be4 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java @@ -23,6 +23,7 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.ml.Model; +import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.trainers.DatasetTrainer; @@ -58,7 +59,7 @@ public static void main(String... args) throws InterruptedException { IgniteCache trainingSet = fillTrainingData(ignite, trainingSetCfg); // Create regression trainer. - DatasetTrainer, Double> trainer = new GDBBinaryClassifierOnTreesTrainer(1.0, 300, 2, 0.); + DatasetTrainer trainer = new GDBBinaryClassifierOnTreesTrainer(1.0, 300, 2, 0.); // Train decision tree model. Model mdl = trainer.fit( diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java index 31dd2b05cb5e1..b2b08d097b73b 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java @@ -23,6 +23,7 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.ml.Model; +import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.trainers.DatasetTrainer; @@ -58,7 +59,7 @@ public static void main(String... args) throws InterruptedException { IgniteCache trainingSet = fillTrainingData(ignite, trainingSetCfg); // Create regression trainer. - DatasetTrainer, Double> trainer = new GDBRegressionOnTreesTrainer(1.0, 2000, 1, 0.); + DatasetTrainer trainer = new GDBRegressionOnTreesTrainer(1.0, 2000, 1, 0.); // Train decision tree model. Model mdl = trainer.fit( diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java index 5b880fcc95ced..2596dbcfb9121 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -72,6 +73,14 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { */ @Override public KMeansModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected KMeansModel updateModel(KMeansModel mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + assert datasetBuilder != null; PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( @@ -85,7 +94,7 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder )) { - final int cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> { + final Integer cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> { if (a == null) return b == null ? 0 : b; if (b == null) @@ -93,7 +102,12 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { return b; }); - centers = initClusterCentersRandomly(dataset, k); + if (cols == null) + return getLastTrainedModelOrThrowEmptyDatasetException(mdl); + + centers = Optional.ofNullable(mdl) + .map(KMeansModel::centers) + .orElseGet(() -> initClusterCentersRandomly(dataset, k)); boolean converged = false; int iteration = 0; @@ -127,6 +141,11 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { return new KMeansModel(centers, distance); } + /** {@inheritDoc} */ + @Override protected boolean checkState(KMeansModel mdl) { + return mdl.centers().length == k && mdl.distanceMeasure().equals(distance); + } + /** * Prepares the data to define new centroids on current iteration. * @@ -281,10 +300,12 @@ TotalCostAndCounts merge(TotalCostAndCounts other) { return this; } + /** + * @return centroid statistics. + */ public ConcurrentHashMap> getCentroidStat() { return centroidStat; } - } /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java index f4397893534fc..493c1da4c6170 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/BaggingModelTrainer.java @@ -177,4 +177,24 @@ private IgniteBiFunction wrapFeatureExtractor( return VectorUtils.of(newFeaturesValues); }); } + + /** + * Learn new models on dataset and create new Compositions over them and already learned models. + * + * @param mdl Learned model. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return New models composition. + */ + @Override public ModelsComposition updateModel(ModelsComposition mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + ArrayList> newModels = new ArrayList<>(mdl.getModels()); + newModels.addAll(fit(datasetBuilder, featureExtractor, lbExtractor).getModels()); + + return new ModelsComposition(newModels, predictionsAggregator); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsComposition.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsComposition.java index e14fa6d3d3d36..36ee62698df30 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsComposition.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsComposition.java @@ -19,6 +19,8 @@ import java.util.Collections; import java.util.List; +import org.apache.ignite.ml.Exportable; +import org.apache.ignite.ml.Exporter; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.predictionsaggregator.PredictionsAggregator; import org.apache.ignite.ml.math.primitives.vector.Vector; @@ -27,7 +29,7 @@ /** * Model consisting of several models and prediction aggregation strategy. */ -public class ModelsComposition implements Model { +public class ModelsComposition implements Model, Exportable { /** * Predictions aggregator. */ @@ -77,6 +79,12 @@ public List> getModels() { return models; } + /** {@inheritDoc} */ + @Override public

      void saveModel(Exporter exporter, P path) { + ModelsCompositionFormat format = new ModelsCompositionFormat(models, predictionsAggregator); + exporter.save(format, path); + } + /** {@inheritDoc} */ @Override public String toString() { return toString(false); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsCompositionFormat.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsCompositionFormat.java new file mode 100644 index 0000000000000..68af0a9b9eacc --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/ModelsCompositionFormat.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.composition; + +import java.io.Serializable; +import java.util.List; +import org.apache.ignite.ml.Model; +import org.apache.ignite.ml.composition.predictionsaggregator.PredictionsAggregator; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * ModelsComposition representation. + * + * @see ModelsComposition + */ +public class ModelsCompositionFormat implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 9115341364082681837L; + + /** Models. */ + private List> models; + + /** Predictions aggregator. */ + private PredictionsAggregator predictionsAggregator; + + /** + * Creates an instance of ModelsCompositionFormat. + * + * @param models Models. + * @param predictionsAggregator Predictions aggregator. + */ + public ModelsCompositionFormat(List> models,PredictionsAggregator predictionsAggregator) { + this.models = models; + this.predictionsAggregator = predictionsAggregator; + } + + /** */ + public List> models() { + return models; + } + + /** */ + public PredictionsAggregator predictionsAggregator() { + return predictionsAggregator; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java index 5a0f52a6023af..c7f21dd1fa362 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java @@ -52,7 +52,7 @@ * * But in practice Decision Trees is most used regressors (see: {@link DecisionTreeRegressionTrainer}). */ -public abstract class GDBTrainer extends DatasetTrainer, Double> { +public abstract class GDBTrainer extends DatasetTrainer { /** Gradient step. */ private final double gradientStep; @@ -81,7 +81,7 @@ public GDBTrainer(double gradStepSize, Integer cntOfIterations, } /** {@inheritDoc} */ - @Override public Model fit(DatasetBuilder datasetBuilder, + @Override public ModelsComposition fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { @@ -119,6 +119,21 @@ public GDBTrainer(double gradStepSize, Integer cntOfIterations, }; } + + //TODO: This method will be implemented in IGNITE-9412 + /** {@inheritDoc} */ + @Override public ModelsComposition updateModel(ModelsComposition mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + throw new UnsupportedOperationException(); + } + + //TODO: This method will be implemented in IGNITE-9412 + /** {@inheritDoc} */ + @Override protected boolean checkState(ModelsComposition mdl) { + throw new UnsupportedOperationException(); + } + /** * Defines unique labels in dataset if need (useful in case of classification). * diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/NNClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/NNClassificationModel.java index b7a57f542350a..d435f91e7c932 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/NNClassificationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/NNClassificationModel.java @@ -174,6 +174,11 @@ protected double getClassVoteForVector(NNStrategy stgy, double distance) { return 1.0; // strategy.SIMPLE } + /** */ + public DistanceMeasure getDistanceMeasure() { + return distanceMeasure; + } + /** {@inheritDoc} */ @Override public int hashCode() { int res = 1; @@ -212,6 +217,17 @@ protected double getClassVoteForVector(NNStrategy stgy, double distance) { .toString(); } + /** + * Sets parameters from other model to this model. + * + * @param mdl Model. + */ + protected void copyParametersFrom(NNClassificationModel mdl) { + this.k = mdl.k; + this.distanceMeasure = mdl.distanceMeasure; + this.stgy = mdl.stgy; + } + /** */ public abstract

      void saveModel(Exporter exporter, P path); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationModel.java index e8c0b4a935db8..bec82a9655bf6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationModel.java @@ -44,12 +44,18 @@ public class ANNClassificationModel extends NNClassificationModel { /** The labeled set of candidates. */ private final LabeledVectorSet candidates; + /** Centroid statistics. */ + private final ANNClassificationTrainer.CentroidStat centroindsStat; + /** * Build the model based on a candidates set. * @param centers The candidates set. + * @param centroindsStat */ - public ANNClassificationModel(LabeledVectorSet centers) { + public ANNClassificationModel(LabeledVectorSet centers, + ANNClassificationTrainer.CentroidStat centroindsStat) { this.candidates = centers; + this.centroindsStat = centroindsStat; } /** */ @@ -57,6 +63,11 @@ public LabeledVectorSet getCandidates() { return candidates; } + /** */ + public ANNClassificationTrainer.CentroidStat getCentroindsStat() { + return centroindsStat; + } + /** {@inheritDoc} */ @Override public Double apply(Vector v) { List neighbors = findKNearestNeighbors(v); @@ -65,7 +76,7 @@ public LabeledVectorSet getCandidates() { /** */ @Override public

      void saveModel(Exporter exporter, P path) { - ANNModelFormat mdlData = new ANNModelFormat(k, distanceMeasure, stgy, candidates); + ANNModelFormat mdlData = new ANNModelFormat(k, distanceMeasure, stgy, candidates, centroindsStat); exporter.save(mdlData, path); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java index 1c45812908797..3e32b6700310c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java @@ -17,9 +17,13 @@ package org.apache.ignite.ml.knn.ann; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.stream.Collectors; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.ml.clustering.kmeans.KMeansModel; import org.apache.ignite.ml.clustering.kmeans.KMeansTrainer; @@ -39,8 +43,8 @@ import org.jetbrains.annotations.NotNull; /** - * ANN algorithm trainer to solve multi-class classification task. - * This trainer is based on ACD strategy and KMeans clustering algorithm to find centroids. + * ANN algorithm trainer to solve multi-class classification task. This trainer is based on ACD strategy and KMeans + * clustering algorithm to find centroids. */ public class ANNClassificationTrainer extends SingleLabelDatasetTrainer { /** Amount of clusters. */ @@ -61,29 +65,55 @@ public class ANNClassificationTrainer extends SingleLabelDatasetTrainer ANNClassificationModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - final Vector[] centers = getCentroids(featureExtractor, lbExtractor, datasetBuilder); + @Override public ANNClassificationModel fit(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - final CentroidStat centroidStat = getCentroidStat(datasetBuilder, featureExtractor, lbExtractor, centers); + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected ANNClassificationModel updateModel(ANNClassificationModel mdl, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + + List centers; + CentroidStat centroidStat; + if (mdl != null) { + centers = Arrays.stream(mdl.getCandidates().data()).map(x -> x.features()).collect(Collectors.toList()); + CentroidStat newStat = getCentroidStat(datasetBuilder, featureExtractor, lbExtractor, centers); + if(newStat == null) + return mdl; + CentroidStat oldStat = mdl.getCentroindsStat(); + centroidStat = newStat.merge(oldStat); + } else { + centers = getCentroids(featureExtractor, lbExtractor, datasetBuilder); + centroidStat = getCentroidStat(datasetBuilder, featureExtractor, lbExtractor, centers); + } final LabeledVectorSet dataset = buildLabelsForCandidates(centers, centroidStat); - return new ANNClassificationModel(dataset); + return new ANNClassificationModel(dataset, centroidStat); + } + + /** {@inheritDoc} */ + @Override protected boolean checkState(ANNClassificationModel mdl) { + return mdl.getDistanceMeasure().equals(distance) && mdl.getCandidates().rowSize() == k; } /** */ - @NotNull private LabeledVectorSet buildLabelsForCandidates(Vector[] centers, CentroidStat centroidStat) { + @NotNull private LabeledVectorSet buildLabelsForCandidates(List centers, + CentroidStat centroidStat) { // init - final LabeledVector[] arr = new LabeledVector[centers.length]; + final LabeledVector[] arr = new LabeledVector[centers.size()]; // fill label for each centroid - for (int i = 0; i < centers.length; i++) - arr[i] = new LabeledVector<>(centers[i], fillProbableLabel(i, centroidStat)); + for (int i = 0; i < centers.size(); i++) + arr[i] = new LabeledVector<>(centers.get(i), fillProbableLabel(i, centroidStat)); return new LabeledVectorSet<>(arr); } @@ -92,13 +122,14 @@ public class ANNClassificationTrainer extends SingleLabelDatasetTrainer Type of a key in {@code upstream} data. - * @param Type of a value in {@code upstream} data. + * @param lbExtractor Label extractor. + * @param datasetBuilder The dataset builder. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. * @return The arrays of vectors. */ - private Vector[] getCentroids(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, DatasetBuilder datasetBuilder) { + private List getCentroids(IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, DatasetBuilder datasetBuilder) { KMeansTrainer trainer = new KMeansTrainer() .withK(k) .withMaxIterations(maxIterations) @@ -112,7 +143,7 @@ private Vector[] getCentroids(IgniteBiFunction featureExtra lbExtractor ); - return mdl.centers(); + return Arrays.asList(mdl.centers()); } /** */ @@ -125,21 +156,24 @@ private ProbableLabel fillProbableLabel(int centroidIdx, CentroidStat centroidSt ConcurrentHashMap centroidLbDistribution = centroidStat.centroidStat().get(centroidIdx); - if(centroidStat.counts.containsKey(centroidIdx)){ + if (centroidStat.counts.containsKey(centroidIdx)) { int clusterSize = centroidStat .counts .get(centroidIdx); clsLbls.keySet().forEach( - (label) -> clsLbls.put(label, centroidLbDistribution.containsKey(label) ? ((double) (centroidLbDistribution.get(label)) / clusterSize) : 0.0) + (label) -> clsLbls.put(label, centroidLbDistribution.containsKey(label) ? ((double)(centroidLbDistribution.get(label)) / clusterSize) : 0.0) ); } return new ProbableLabel(clsLbls); } /** */ - private CentroidStat getCentroidStat(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, Vector[] centers) { + private CentroidStat getCentroidStat(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, List centers) { + PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( featureExtractor, lbExtractor @@ -174,7 +208,7 @@ private CentroidStat getCentroidStat(DatasetBuilder datasetBuilder, } res.counts.merge(centroidIdx, 1, - (IgniteBiFunction) (i1, i2) -> i1 + i2); + (IgniteBiFunction)(i1, i2) -> i1 + i2); } return res; }, (a, b) -> { @@ -194,15 +228,15 @@ private CentroidStat getCentroidStat(DatasetBuilder datasetBuilder, * Find the closest cluster center index and distance to it from a given point. * * @param centers Centers to look in. - * @param pnt Point. + * @param pnt Point. */ - private IgniteBiTuple findClosestCentroid(Vector[] centers, LabeledVector pnt) { + private IgniteBiTuple findClosestCentroid(List centers, LabeledVector pnt) { double bestDistance = Double.POSITIVE_INFINITY; int bestInd = 0; - for (int i = 0; i < centers.length; i++) { - if (centers[i] != null) { - double dist = distance.compute(centers[i], pnt.features()); + for (int i = 0; i < centers.size(); i++) { + if (centers.get(i) != null) { + double dist = distance.compute(centers.get(i), pnt.features()); if (dist < bestDistance) { bestDistance = dist; bestInd = i; @@ -212,7 +246,6 @@ private IgniteBiTuple findClosestCentroid(Vector[] centers, Lab return new IgniteBiTuple<>(bestInd, bestDistance); } - /** * Gets the amount of clusters. * @@ -314,7 +347,9 @@ public ANNClassificationTrainer withSeed(long seed) { } /** Service class used for statistics. */ - public static class CentroidStat { + public static class CentroidStat implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 7624883170532045144L; /** Count of points closest to the center with a given index. */ ConcurrentHashMap> centroidStat = new ConcurrentHashMap<>(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNModelFormat.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNModelFormat.java index e10f3b22dc408..be09828df3781 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNModelFormat.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNModelFormat.java @@ -30,6 +30,9 @@ * @see ANNClassificationModel */ public class ANNModelFormat extends KNNModelFormat implements Serializable { + /** Centroid statistics. */ + private final ANNClassificationTrainer.CentroidStat candidatesStat; + /** The labeled set of candidates. */ private LabeledVectorSet candidates; @@ -38,15 +41,18 @@ public class ANNModelFormat extends KNNModelFormat implements Serializable { * @param k Amount of nearest neighbors. * @param measure Distance measure. * @param stgy kNN strategy. + * @param candidatesStat */ public ANNModelFormat(int k, - DistanceMeasure measure, - NNStrategy stgy, - LabeledVectorSet candidates) { + DistanceMeasure measure, + NNStrategy stgy, + LabeledVectorSet candidates, + ANNClassificationTrainer.CentroidStat candidatesStat) { this.k = k; this.distanceMeasure = measure; this.stgy = stgy; this.candidates = candidates; + this.candidatesStat = candidatesStat; } /** */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java index 0b88f8181cf4f..0d03ee5bbaef6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -42,25 +43,29 @@ public class KNNClassificationModel extends NNClassificationModel implements Exp /** */ private static final long serialVersionUID = -127386523291350345L; - /** Dataset. */ - private Dataset> dataset; + /** Datasets. */ + private List>> datasets; /** * Builds the model via prepared dataset. + * * @param dataset Specially prepared object to run algorithm over it. */ public KNNClassificationModel(Dataset> dataset) { - this.dataset = dataset; + this.datasets = new ArrayList<>(); + if (dataset != null) + datasets.add(dataset); } /** {@inheritDoc} */ @Override public Double apply(Vector v) { - if(dataset != null) { + if (!datasets.isEmpty()) { List neighbors = findKNearestNeighbors(v); return classify(neighbors, v, stgy); - } else + } else { throw new IllegalStateException("The train kNN dataset is null"); + } } /** */ @@ -77,6 +82,17 @@ public KNNClassificationModel(Dataset findKNearestNeighbors(Vector v) { + List neighborsFromPartitions = datasets.stream() + .flatMap(dataset -> findKNearestNeighborsInDataset(v, dataset).stream()) + .collect(Collectors.toList()); + + LabeledVectorSet neighborsToFilter = buildLabeledDatasetOnListOfVectors(neighborsFromPartitions); + + return Arrays.asList(getKClosestVectors(neighborsToFilter, getDistances(v, neighborsToFilter))); + } + + private List findKNearestNeighborsInDataset(Vector v, + Dataset> dataset) { List neighborsFromPartitions = dataset.compute(data -> { TreeMap> distanceIdxPairs = getDistances(v, data); return Arrays.asList(getKClosestVectors(data, distanceIdxPairs)); @@ -88,12 +104,14 @@ protected List findKNearestNeighbors(Vector v) { return Stream.concat(a.stream(), b.stream()).collect(Collectors.toList()); }); + if(neighborsFromPartitions == null) + return Collections.emptyList(); + LabeledVectorSet neighborsToFilter = buildLabeledDatasetOnListOfVectors(neighborsFromPartitions); return Arrays.asList(getKClosestVectors(neighborsToFilter, getDistances(v, neighborsToFilter))); } - /** */ private double classify(List neighbors, Vector v, NNStrategy stgy) { Map clsVotes = new HashMap<>(); @@ -116,5 +134,13 @@ private double classify(List neighbors, Vector v, NNStrategy stgy return getClassWithMaxVotes(clsVotes); } - + /** + * Copy parameters from other model and save all datasets from it. + * + * @param model Model. + */ + public void copyStateFrom(KNNClassificationModel model) { + this.copyParametersFrom(model); + datasets.addAll(model.datasets); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java index e0a81f9b432b2..1a3ff7307a48d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java @@ -37,6 +37,24 @@ public class KNNClassificationTrainer extends SingleLabelDatasetTrainer KNNClassificationModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - return new KNNClassificationModel(KNNUtils.buildDataset(datasetBuilder, featureExtractor, lbExtractor)); + + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override public KNNClassificationModel updateModel(KNNClassificationModel mdl, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + + KNNClassificationModel res = new KNNClassificationModel(KNNUtils.buildDataset(datasetBuilder, + featureExtractor, lbExtractor)); + if (mdl != null) + res.copyStateFrom(mdl); + return res; + } + + /** {@inheritDoc} */ + @Override protected boolean checkState(KNNClassificationModel mdl) { + return true; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java index 395ce613b7542..7a42dc8169852 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java @@ -37,6 +37,23 @@ public class KNNRegressionTrainer extends SingleLabelDatasetTrainer KNNRegressionModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - return new KNNRegressionModel(KNNUtils.buildDataset(datasetBuilder, featureExtractor, lbExtractor)); + + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override public KNNRegressionModel updateModel(KNNRegressionModel mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + KNNRegressionModel res = new KNNRegressionModel(KNNUtils.buildDataset(datasetBuilder, + featureExtractor, lbExtractor)); + if (mdl != null) + res.copyStateFrom(mdl); + return res; + } + + /** {@inheritDoc} */ + @Override protected boolean checkState(KNNRegressionModel mdl) { + return true; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java index 7a362f7b1fd45..c9281c076784b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java @@ -78,7 +78,9 @@ public abstract class AbstractLSQR { */ public LSQRResult solve(double damp, double atol, double btol, double conlim, double iterLim, boolean calcVar, double[] x0) { - int n = getColumns(); + Integer n = getColumns(); + if(n == null) + return null; if (iterLim < 0) iterLim = 2 * n; @@ -313,7 +315,7 @@ public LSQRResult solve(double damp, double atol, double btol, double conlim, do protected abstract double[] iter(double bnorm, double[] target); /** */ - protected abstract int getColumns(); + protected abstract Integer getColumns(); /** */ private static double[] symOrtho(double a, double b) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java index f75caefab76a4..14356e1eed877 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java @@ -100,7 +100,7 @@ else if (b == null) * * @return number of columns */ - @Override protected int getColumns() { + @Override protected Integer getColumns() { return dataset.compute( data -> data.getFeatures() == null ? null : data.getFeatures().length / data.getRows(), (a, b) -> { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java index 6727ba90bc73b..8f1a4cb8e8159 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java @@ -111,12 +111,25 @@ public MLPTrainer(IgniteFunction public MultilayerPerceptron fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected MultilayerPerceptron updateModel(MultilayerPerceptron lastLearnedModel, + DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + try (Dataset dataset = datasetBuilder.build( new EmptyContextBuilder<>(), new SimpleLabeledDatasetDataBuilder<>(featureExtractor, lbExtractor) )) { - MLPArchitecture arch = archSupplier.apply(dataset); - MultilayerPerceptron mdl = new MultilayerPerceptron(arch, new RandomInitializer(seed)); + MultilayerPerceptron mdl; + if (lastLearnedModel != null) { + mdl = lastLearnedModel; + } else { + MLPArchitecture arch = archSupplier.apply(dataset); + mdl = new MultilayerPerceptron(arch, new RandomInitializer(seed)); + } ParameterUpdateCalculator updater = updatesStgy.getUpdatesCalculator(); for (int i = 0; i < maxIterations; i += locIterations) { @@ -178,6 +191,9 @@ else if (b == null) } ); + if (totUp == null) + return getLastTrainedModelOrThrowEmptyDatasetException(lastLearnedModel); + P update = updatesStgy.allUpdatesReducer().apply(totUp); mdl = updater.update(mdl, update); } @@ -189,6 +205,11 @@ else if (b == null) } } + /** {@inheritDoc} */ + @Override protected boolean checkState(MultilayerPerceptron mdl) { + return true; + } + /** * Builds a batch of the data by fetching specified rows. * diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java index 1886ee557da75..b977864d51e2f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java @@ -17,6 +17,7 @@ package org.apache.ignite.ml.preprocessing; +import java.util.Map; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.ml.dataset.DatasetBuilder; @@ -24,8 +25,6 @@ import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import java.util.Map; - /** * Trainer for preprocessor. * diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java index 8197779891c51..5497177a30bb6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java @@ -38,16 +38,34 @@ public class LinearRegressionLSQRTrainer extends SingleLabelDatasetTrainer LinearRegressionModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected LinearRegressionModel updateModel(LinearRegressionModel mdl, + DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + LSQRResult res; try (LSQROnHeap lsqr = new LSQROnHeap<>( datasetBuilder, new SimpleLabeledDatasetDataBuilder<>( new FeatureExtractorWrapper<>(featureExtractor), - lbExtractor.andThen(e -> new double[]{e}) + lbExtractor.andThen(e -> new double[] {e}) ) )) { - res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null); + double[] x0 = null; + if (mdl != null) { + int x0Size = mdl.getWeights().size() + 1; + Vector weights = mdl.getWeights().like(x0Size); + mdl.getWeights().nonZeroes().forEach(ith -> weights.set(ith.index(), ith.get())); + weights.set(weights.size() - 1, mdl.getIntercept()); + x0 = weights.asArray(); + } + res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, x0); + if (res == null) + return getLastTrainedModelOrThrowEmptyDatasetException(mdl); } catch (Exception e) { throw new RuntimeException(e); @@ -58,4 +76,9 @@ public class LinearRegressionLSQRTrainer extends SingleLabelDatasetTrainer extends SingleLa private final UpdatesStrategy updatesStgy; /** Max number of iteration. */ - private final int maxIterations; + private int maxIterations = 1000; /** Batch size. */ - private final int batchSize; + private int batchSize = 10; /** Number of local iterations. */ - private final int locIterations; + private int locIterations = 100; /** Seed for random generator. */ - private final long seed; + private long seed = System.currentTimeMillis(); /** * Constructs a new instance of linear regression SGD trainer. @@ -72,10 +74,24 @@ public LinearRegressionSGDTrainer(UpdatesStrategy updatesStgy) { + this.updatesStgy = updatesStgy; + } + /** {@inheritDoc} */ @Override public LinearRegressionModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected LinearRegressionModel updateModel(LinearRegressionModel mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + IgniteFunction, MLPArchitecture> archSupplier = dataset -> { int cols = dataset.compute(data -> { @@ -108,7 +124,10 @@ public LinearRegressionSGDTrainer(UpdatesStrategy lbE = (IgniteBiFunction)(k, v) -> new double[] {lbExtractor.apply(k, v)}; - MultilayerPerceptron mlp = trainer.fit(datasetBuilder, featureExtractor, lbE); + MultilayerPerceptron mlp = Optional.ofNullable(mdl) + .map(this::restoreMLPState) + .map(m -> trainer.update(m, datasetBuilder, featureExtractor, lbE)) + .orElseGet(() -> trainer.fit(datasetBuilder, featureExtractor, lbE)); double[] p = mlp.parameters().getStorage().data(); @@ -117,4 +136,72 @@ public LinearRegressionSGDTrainer(UpdatesStrategy mlpState.set(ith.index(), ith.get())); + mlpState.set(mlpState.size() - 1, intercept); + perceptron.setParameters(mlpState); + return perceptron; + } + + /** {@inheritDoc} */ + @Override protected boolean checkState(LinearRegressionModel mdl) { + return true; + } + + /** + * Set up the max number of iterations before convergence. + * + * @param maxIterations The parameter value. + * @return Model with new max number of iterations before convergence parameter value. + */ + public LinearRegressionSGDTrainer

      withMaxIterations(int maxIterations) { + this.maxIterations = maxIterations; + return this; + } + + /** + * Set up the batchSize parameter. + * + * @param batchSize The size of learning batch. + * @return Trainer with new batch size parameter value. + */ + public LinearRegressionSGDTrainer

      withBatchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + /** + * Set up the amount of local iterations of SGD algorithm. + * + * @param amountOfLocIterations The parameter value. + * @return Trainer with new locIterations parameter value. + */ + public LinearRegressionSGDTrainer

      withLocIterations(int amountOfLocIterations) { + this.locIterations = amountOfLocIterations; + return this; + } + + /** + * Set up the random seed parameter. + * + * @param seed Seed for random generator. + * @return Trainer with new seed parameter value. + */ + public LinearRegressionSGDTrainer

      withSeed(long seed) { + this.seed = seed; + return this; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java index 639627950ac70..839dab5582007 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java @@ -34,6 +34,7 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture; import org.apache.ignite.ml.optimization.LossFunctions; import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; +import org.jetbrains.annotations.NotNull; /** * Trainer of the logistic regression model based on stochastic gradient descent algorithm. @@ -76,8 +77,15 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy LogisticRegressionModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - IgniteFunction, MLPArchitecture> archSupplier = dataset -> { + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected LogisticRegressionModel updateModel(LogisticRegressionModel mdl, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + IgniteFunction, MLPArchitecture> archSupplier = dataset -> { int cols = dataset.compute(data -> { if (data.getFeatures() == null) return null; @@ -106,7 +114,13 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy new double[] {lbExtractor.apply(k, v)}); + IgniteBiFunction lbExtractorWrapper = (k, v) -> new double[] {lbExtractor.apply(k, v)}; + MultilayerPerceptron mlp; + if(mdl != null) { + mlp = restoreMLPState(mdl); + mlp = trainer.update(mlp, datasetBuilder, featureExtractor, lbExtractorWrapper); + } else + mlp = trainer.fit(datasetBuilder, featureExtractor, lbExtractorWrapper); double[] params = mlp.parameters().getStorage().data(); @@ -114,4 +128,28 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy mlpState.set(ith.index(), ith.get())); + mlpState.set(mlpState.size() - 1, intercept); + perceptron.setParameters(mlpState); + return perceptron; + } + + /** {@inheritDoc} */ + @Override protected boolean checkState(LogisticRegressionModel mdl) { + return true; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassModel.java index 56d2d29d539e6..a7c9118bb0b1f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassModel.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.TreeMap; import org.apache.ignite.ml.Exportable; import org.apache.ignite.ml.Exporter; @@ -103,4 +104,12 @@ public LogRegressionMultiClassModel() { public void add(double clsLb, LogisticRegressionModel mdl) { models.put(clsLb, mdl); } + + /** + * @param clsLb Class label. + * @return model for class label if it exists. + */ + public Optional getModel(Double clsLb) { + return Optional.ofNullable(models.get(clsLb)); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java index 4885373ae094f..eb44301e4ecd7 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java @@ -22,6 +22,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -33,6 +34,7 @@ import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.nn.MultilayerPerceptron; import org.apache.ignite.ml.nn.UpdatesStrategy; +import org.apache.ignite.ml.regressions.logistic.binomial.LogisticRegressionModel; import org.apache.ignite.ml.regressions.logistic.binomial.LogisticRegressionSGDTrainer; import org.apache.ignite.ml.structures.partition.LabelPartitionDataBuilderOnHeap; import org.apache.ignite.ml.structures.partition.LabelPartitionDataOnHeap; @@ -71,6 +73,19 @@ public class LogRegressionMultiClassTrainer

      IgniteBiFunction lbExtractor) { List classes = extractClassLabels(datasetBuilder, lbExtractor); + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override public LogRegressionMultiClassModel updateModel(LogRegressionMultiClassModel mdl, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + + List classes = extractClassLabels(datasetBuilder, lbExtractor); + + if(classes.isEmpty()) + return getLastTrainedModelOrThrowEmptyDatasetException(mdl); + LogRegressionMultiClassModel multiClsMdl = new LogRegressionMultiClassModel(); classes.forEach(clsLb -> { @@ -85,12 +100,23 @@ public class LogRegressionMultiClassTrainer

      else return 0.0; }; - multiClsMdl.add(clsLb, trainer.fit(datasetBuilder, featureExtractor, lbTransformer)); + + LogisticRegressionModel model = Optional.ofNullable(mdl) + .flatMap(multiClassModel -> multiClassModel.getModel(clsLb)) + .map(learnedModel -> trainer.update(learnedModel, datasetBuilder, featureExtractor, lbTransformer)) + .orElseGet(() -> trainer.fit(datasetBuilder, featureExtractor, lbTransformer)); + + multiClsMdl.add(clsLb, model); }); return multiClsMdl; } + /** {@inheritDoc} */ + @Override protected boolean checkState(LogRegressionMultiClassModel mdl) { + return true; + } + /** Iterates among dataset and collects class labels. */ private List extractClassLabels(DatasetBuilder datasetBuilder, IgniteBiFunction lbExtractor) { @@ -121,7 +147,8 @@ private List extractClassLabels(DatasetBuilder datasetBuild return Stream.of(a, b).flatMap(Collection::stream).collect(Collectors.toSet()); }); - res.addAll(clsLabels); + if (clsLabels != null) + res.addAll(clsLabels); } catch (Exception e) { @@ -191,7 +218,7 @@ public LogRegressionMultiClassTrainer withAmountOfLocIterations(int amountOfLocI } /** - * Set up the regularization parameter. + * Set up the random seed parameter. * * @param seed Seed for random generator. * @return Trainer with new seed parameter value. diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java index 933a7128731be..573df1a3395df 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java @@ -22,9 +22,11 @@ import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.StorageConstants; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.impl.DenseVector; +import org.apache.ignite.ml.math.primitives.vector.impl.SparseVector; import org.apache.ignite.ml.structures.LabeledVector; import org.apache.ignite.ml.structures.LabeledVectorSet; import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap; @@ -61,6 +63,14 @@ public class SVMLinearBinaryClassificationTrainer extends SingleLabelDatasetTrai @Override public SVMLinearBinaryClassificationModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected SVMLinearBinaryClassificationModel updateModel(SVMLinearBinaryClassificationModel mdl, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + assert datasetBuilder != null; PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( @@ -74,29 +84,57 @@ public class SVMLinearBinaryClassificationTrainer extends SingleLabelDatasetTrai (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder )) { - final int cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> { - if (a == null) - return b == null ? 0 : b; - if (b == null) - return a; - return b; - }); - - final int weightVectorSizeWithIntercept = cols + 1; - - weights = initializeWeightsWithZeros(weightVectorSizeWithIntercept); + if (mdl == null) { + final int cols = dataset.compute(org.apache.ignite.ml.structures.Dataset::colSize, (a, b) -> { + if (a == null) + return b == null ? 0 : b; + if (b == null) + return a; + return b; + }); + + final int weightVectorSizeWithIntercept = cols + 1; + weights = initializeWeightsWithZeros(weightVectorSizeWithIntercept); + } else { + weights = getStateVector(mdl); + } for (int i = 0; i < this.getAmountOfIterations(); i++) { Vector deltaWeights = calculateUpdates(weights, dataset); + if (deltaWeights == null) + return getLastTrainedModelOrThrowEmptyDatasetException(mdl); + weights = weights.plus(deltaWeights); // creates new vector } - } - catch (Exception e) { + } catch (Exception e) { throw new RuntimeException(e); } return new SVMLinearBinaryClassificationModel(weights.viewPart(1, weights.size() - 1), weights.get(0)); } + /** {@inheritDoc} */ + @Override protected boolean checkState(SVMLinearBinaryClassificationModel mdl) { + return true; + } + + /** + * @param mdl Model. + * @return vector of model weights with intercept. + */ + private Vector getStateVector(SVMLinearBinaryClassificationModel mdl) { + double intercept = mdl.intercept(); + Vector weights = mdl.weights(); + + int stateVectorSize = weights.size() + 1; + Vector result = weights.isDense() ? + new DenseVector(stateVectorSize) : + new SparseVector(stateVectorSize, StorageConstants.RANDOM_ACCESS_MODE); + + result.set(0, intercept); + weights.nonZeroes().forEach(ith -> result.set(ith.index(), ith.get())); + return result; + } + /** */ @NotNull private Vector initializeWeightsWithZeros(int vectorSize) { return new DenseVector(vectorSize); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java index 4b04824206cbc..46bf4b2976c70 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.TreeMap; import org.apache.ignite.ml.Exportable; import org.apache.ignite.ml.Exporter; @@ -102,4 +103,12 @@ public SVMLinearMultiClassClassificationModel() { public void add(double clsLb, SVMLinearBinaryClassificationModel mdl) { models.put(clsLb, mdl); } + + /** + * @param clsLb Class label. + * @return model trained for target class if it exists. + */ + public Optional getModelForClass(double clsLb) { + return Optional.of(models.get(clsLb)); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java index 4b7cc95ebe1c8..b77baa2a1f84e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java @@ -57,15 +57,26 @@ public class SVMLinearMultiClassClassificationTrainer /** * Trains model based on the specified data. * - * @param datasetBuilder Dataset builder. + * @param datasetBuilder Dataset builder. * @param featureExtractor Feature extractor. - * @param lbExtractor Label extractor. + * @param lbExtractor Label extractor. * @return Model. */ @Override public SVMLinearMultiClassClassificationModel fit(DatasetBuilder datasetBuilder, - IgniteBiFunction featureExtractor, - IgniteBiFunction lbExtractor) { + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override public SVMLinearMultiClassClassificationModel updateModel( + SVMLinearMultiClassClassificationModel mdl, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + List classes = extractClassLabels(datasetBuilder, lbExtractor); + if (classes.isEmpty()) + return getLastTrainedModelOrThrowEmptyDatasetException(mdl); SVMLinearMultiClassClassificationModel multiClsMdl = new SVMLinearMultiClassClassificationModel(); @@ -84,14 +95,60 @@ public class SVMLinearMultiClassClassificationTrainer else return -1.0; }; - multiClsMdl.add(clsLb, trainer.fit(datasetBuilder, featureExtractor, lbTransformer)); + + SVMLinearBinaryClassificationModel model; + if (mdl == null) + model = learnNewModel(trainer, datasetBuilder, featureExtractor, lbTransformer); + else + model = updateModel(mdl, clsLb, trainer, datasetBuilder, featureExtractor, lbTransformer); + multiClsMdl.add(clsLb, model); }); return multiClsMdl; } + /** {@inheritDoc} */ + @Override protected boolean checkState(SVMLinearMultiClassClassificationModel mdl) { + return true; + } + + /** + * Trains model based on the specified data. + * + * @param svmTrainer Prepared SVM trainer. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + */ + private SVMLinearBinaryClassificationModel learnNewModel(SVMLinearBinaryClassificationTrainer svmTrainer, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + + return svmTrainer.fit(datasetBuilder, featureExtractor, lbExtractor); + } + + /** + * Updates already learned model or fit new model if there is no model for current class label. + * + * @param multiClsMdl Learning multi-class model. + * @param clsLb Current class label. + * @param svmTrainer Prepared SVM trainer. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + */ + private SVMLinearBinaryClassificationModel updateModel(SVMLinearMultiClassClassificationModel multiClsMdl, + Double clsLb, SVMLinearBinaryClassificationTrainer svmTrainer, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + return multiClsMdl.getModelForClass(clsLb) + .map(learnedModel -> svmTrainer.update(learnedModel, datasetBuilder, featureExtractor, lbExtractor)) + .orElseGet(() -> svmTrainer.fit(datasetBuilder, featureExtractor, lbExtractor)); + } + /** Iterates among dataset and collects class labels. */ - private List extractClassLabels(DatasetBuilder datasetBuilder, IgniteBiFunction lbExtractor) { + private List extractClassLabels(DatasetBuilder datasetBuilder, + IgniteBiFunction lbExtractor) { assert datasetBuilder != null; PartitionDataBuilder partDataBuilder = new LabelPartitionDataBuilderOnHeap<>(lbExtractor); @@ -107,7 +164,8 @@ private List extractClassLabels(DatasetBuilder datasetBuild final double[] lbs = data.getY(); - for (double lb : lbs) locClsLabels.add(lb); + for (double lb : lbs) + locClsLabels.add(lb); return locClsLabels; }, (a, b) -> { @@ -118,8 +176,8 @@ private List extractClassLabels(DatasetBuilder datasetBuild return Stream.of(a, b).flatMap(Collection::stream).collect(Collectors.toSet()); }); - res.addAll(clsLabels); - + if (clsLabels != null) + res.addAll(clsLabels); } catch (Exception e) { throw new RuntimeException(e); } @@ -132,7 +190,7 @@ private List extractClassLabels(DatasetBuilder datasetBuild * @param lambda The regularization parameter. Should be more than 0.0. * @return Trainer with new lambda parameter value. */ - public SVMLinearMultiClassClassificationTrainer withLambda(double lambda) { + public SVMLinearMultiClassClassificationTrainer withLambda(double lambda) { assert lambda > 0.0; this.lambda = lambda; return this; @@ -162,7 +220,7 @@ public int amountOfIterations() { * @param amountOfIterations The parameter value. * @return Trainer with new amountOfIterations parameter value. */ - public SVMLinearMultiClassClassificationTrainer withAmountOfIterations(int amountOfIterations) { + public SVMLinearMultiClassClassificationTrainer withAmountOfIterations(int amountOfIterations) { this.amountOfIterations = amountOfIterations; return this; } @@ -182,7 +240,7 @@ public int amountOfLocIterations() { * @param amountOfLocIterations The parameter value. * @return Trainer with new amountOfLocIterations parameter value. */ - public SVMLinearMultiClassClassificationTrainer withAmountOfLocIterations(int amountOfLocIterations) { + public SVMLinearMultiClassClassificationTrainer withAmountOfLocIterations(int amountOfLocIterations) { this.amountOfLocIterations = amountOfLocIterations; return this; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java index 2f5d5d60d8475..fb34c93f0e7e0 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java @@ -26,8 +26,10 @@ import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.environment.LearningEnvironment; +import org.apache.ignite.ml.environment.logging.MLLogger; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.jetbrains.annotations.NotNull; /** * Interface for trainers. Trainer is just a function which produces model from the data. @@ -52,6 +54,71 @@ public abstract class DatasetTrainer { public abstract M fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor); + /** + * Gets state of model in arguments, compare it with training parameters of trainer and if they are fit then + * trainer updates model in according to new data and return new model. In other case trains new model. + * + * @param mdl Learned model. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Updated model. + */ + public M update(M mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + if(mdl != null) { + if(checkState(mdl)) { + return updateModel(mdl, datasetBuilder, featureExtractor, lbExtractor); + } else { + environment.logger(getClass()).log( + MLLogger.VerboseLevel.HIGH, + "Model cannot be updated because of initial state of " + + "it doesn't corresponds to trainer parameters" + ); + } + } + + return fit(datasetBuilder, featureExtractor, lbExtractor); + } + + /** + * @param mdl Model. + * @return true if current critical for training parameters correspond to parameters from last training. + */ + protected abstract boolean checkState(M mdl); + + /** + * Used on update phase when given dataset is empty. + * If last trained model exist then method returns it. In other case throws IllegalArgumentException. + * + * @param lastTrainedMdl Model. + */ + @NotNull protected M getLastTrainedModelOrThrowEmptyDatasetException(M lastTrainedMdl) { + String msg = "Cannot train model on empty dataset"; + if (lastTrainedMdl != null) { + environment.logger(getClass()).log(MLLogger.VerboseLevel.HIGH, msg); + return lastTrainedMdl; + } else + throw new EmptyDatasetException(); + } + + /** + * Gets state of model in arguments, update in according to new data and return new model. + * + * @param mdl Learned model. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Updated model. + */ + protected abstract M updateModel(M mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor); + /** * Trains model based on the specified data. * @@ -72,6 +139,27 @@ public M fit(Ignite ignite, IgniteCache cache, ); } + /** + * Gets state of model in arguments, update in according to new data and return new model. + * + * @param mdl Learned model. + * @param ignite Ignite instance. + * @param cache Ignite cache. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Updated model. + */ + public M update(M mdl, Ignite ignite, IgniteCache cache, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return update( + mdl, new CacheBasedDatasetBuilder<>(ignite, cache), + featureExtractor, + lbExtractor + ); + } + /** * Trains model based on the specified data. * @@ -93,6 +181,28 @@ public M fit(Ignite ignite, IgniteCache cache, IgniteBiPredicate Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Updated model. + */ + public M update(M mdl, Ignite ignite, IgniteCache cache, IgniteBiPredicate filter, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return update( + mdl, new CacheBasedDatasetBuilder<>(ignite, cache, filter), + featureExtractor, + lbExtractor + ); + } + /** * Trains model based on the specified data. * @@ -113,6 +223,27 @@ public M fit(Map data, int parts, IgniteBiFunction fe ); } + /** + * Gets state of model in arguments, update in according to new data and return new model. + * + * @param mdl Learned model. + * @param data Data. + * @param parts Number of partitions. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Updated model. + */ + public M update(M mdl, Map data, int parts, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + return update( + mdl, new LocalDatasetBuilder<>(data, parts), + featureExtractor, + lbExtractor + ); + } + /** * Trains model based on the specified data. * @@ -135,6 +266,28 @@ public M fit(Map data, IgniteBiPredicate filter, int parts, ); } + /** + * Gets state of model in arguments, update in according to new data and return new model. + * + * @param data Data. + * @param filter Filter for {@code upstream} data. + * @param parts Number of partitions. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Updated model. + */ + public M update(M mdl, Map data, IgniteBiPredicate filter, int parts, + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + return update( + mdl, new LocalDatasetBuilder<>(data, filter, parts), + featureExtractor, + lbExtractor + ); + } + /** * Sets learning Environment * @param environment Environment. @@ -142,4 +295,17 @@ public M fit(Map data, IgniteBiPredicate filter, int parts, public void setEnvironment(LearningEnvironment environment) { this.environment = environment; } + + /** */ + public static class EmptyDatasetException extends IllegalArgumentException { + /** Serial version uid. */ + private static final long serialVersionUID = 6914650522523293521L; + + /** + * Constructs an instance of EmptyDatasetException. + */ + public EmptyDatasetException() { + super("Cannot train model on empty dataset"); + } + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java index de8994a6abbd8..355048a1e4937 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java @@ -86,6 +86,29 @@ public abstract class DecisionTree> extends Dataset } } + /** + * Trains new model based on dataset because there is no valid approach to update decision trees. + * + * @param mdl Learned model. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return New model based on new dataset. + */ + @Override public DecisionTreeNode updateModel(DecisionTreeNode mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + return fit(datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected boolean checkState(DecisionTreeNode mdl) { + return true; + } + + /** */ public DecisionTreeNode fit(Dataset dataset) { return split(dataset, e -> true, 0, getImpurityMeasureCalculator(dataset)); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java index 559dffff1c9e8..7832584f65374 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainer.java @@ -64,8 +64,9 @@ public RandomForestClassifierTrainer(List meta) { * This id can be used as index in arrays or lists. * * @param dataset Dataset. + * @return true if initialization was done. */ - @Override protected void init(Dataset dataset) { + @Override protected boolean init(Dataset dataset) { Set uniqLabels = dataset.compute( x -> { Set labels = new HashSet<>(); @@ -85,11 +86,14 @@ public RandomForestClassifierTrainer(List meta) { } ); + if(uniqLabels == null) + return false; + int i = 0; for (Double label : uniqLabels) lblMapping.put(label, i++); - super.init(dataset); + return super.init(dataset); } /** {@inheritDoc} */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java index cb25aa32df1ad..91fcf0ab303ca 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java @@ -30,6 +30,7 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; @@ -116,7 +117,8 @@ public RandomForestTrainer(List meta) { new EmptyContextBuilder<>(), new BootstrappedDatasetBuilder<>(featureExtractor, lbExtractor, cntOfTrees, subsampleSize))) { - init(dataset); + if(!init(dataset)) + return buildComposition(Collections.emptyList()); models = fit(dataset); } catch (Exception e) { @@ -202,7 +204,8 @@ public T withSeed(long seed) { * * @param dataset Dataset. */ - protected void init(Dataset dataset) { + protected boolean init(Dataset dataset) { + return true; } /** @@ -215,6 +218,8 @@ private List fit(Dataset d Queue treesQueue = createRootsQueue(); ArrayList roots = initTrees(treesQueue); Map histMeta = computeHistogramMeta(meta, dataset); + if(histMeta.isEmpty()) + return Collections.emptyList(); ImpurityHistogramsComputer histogramsComputer = createImpurityHistogramsComputer(); while (!treesQueue.isEmpty()) { @@ -232,6 +237,23 @@ private List fit(Dataset d return roots; } + /** {@inheritDoc} */ + @Override protected boolean checkState(ModelsComposition mdl) { + ModelsComposition fakeComposition = buildComposition(Collections.emptyList()); + return mdl.getPredictionsAggregator().getClass() == fakeComposition.getPredictionsAggregator().getClass(); + } + + /** {@inheritDoc} */ + @Override protected ModelsComposition updateModel(ModelsComposition mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + ArrayList> oldModels = new ArrayList<>(mdl.getModels()); + ModelsComposition newModels = fit(datasetBuilder, featureExtractor, lbExtractor); + oldModels.addAll(newModels.getModels()); + + return new ModelsComposition(oldModels, mdl.getPredictionsAggregator()); + } + /** * Split node with NodeId if need. * @@ -302,6 +324,8 @@ private Map computeHistogramMeta(List meta, List stats = new NormalDistributionStatisticsComputer() .computeStatistics(meta, dataset); + if(stats == null) + return Collections.emptyMap(); Map bucketsMeta = new HashMap<>(); for (int i = 0; i < stats.size(); i++) { diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java index aae5af1949a50..03f044a23a59f 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java @@ -27,6 +27,7 @@ import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.math.primitives.vector.impl.DenseVector; +import org.jetbrains.annotations.NotNull; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -39,19 +40,70 @@ public class KMeansTrainerTest { /** Precision in test checks. */ private static final double PRECISION = 1e-2; + /** Data. */ + private static final Map data = new HashMap<>(); + + static { + data.put(0, new double[] {1.0, 1.0, 1.0}); + data.put(1, new double[] {1.0, 2.0, 1.0}); + data.put(2, new double[] {2.0, 1.0, 1.0}); + data.put(3, new double[] {-1.0, -1.0, 2.0}); + data.put(4, new double[] {-1.0, -2.0, 2.0}); + data.put(5, new double[] {-2.0, -1.0, 2.0}); + } + /** * A few points, one cluster, one iteration */ @Test public void findOneClusters() { - Map data = new HashMap<>(); - data.put(0, new double[]{1.0, 1.0, 1.0}); - data.put(1, new double[]{1.0, 2.0, 1.0}); - data.put(2, new double[]{2.0, 1.0, 1.0}); - data.put(3, new double[]{-1.0, -1.0, 2.0}); - data.put(4, new double[]{-1.0, -2.0, 2.0}); - data.put(5, new double[]{-2.0, -1.0, 2.0}); + KMeansTrainer trainer = createAndCheckTrainer(); + KMeansModel knnMdl = trainer.withK(1).fit( + new LocalDatasetBuilder<>(data, 2), + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + Vector firstVector = new DenseVector(new double[] {2.0, 2.0}); + assertEquals(knnMdl.apply(firstVector), 0.0, PRECISION); + Vector secondVector = new DenseVector(new double[] {-2.0, -2.0}); + assertEquals(knnMdl.apply(secondVector), 0.0, PRECISION); + assertEquals(trainer.getMaxIterations(), 1); + assertEquals(trainer.getEpsilon(), PRECISION, PRECISION); + } + /** */ + @Test + public void testUpdateMdl() { + KMeansTrainer trainer = createAndCheckTrainer(); + KMeansModel originalMdl = trainer.withK(1).fit( + new LocalDatasetBuilder<>(data, 2), + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + KMeansModel updatedMdlOnSameDataset = trainer.update( + originalMdl, + new LocalDatasetBuilder<>(data, 2), + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + KMeansModel updatedMdlOnEmptyDataset = trainer.update( + originalMdl, + new LocalDatasetBuilder<>(new HashMap(), 2), + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + Vector firstVector = new DenseVector(new double[] {2.0, 2.0}); + Vector secondVector = new DenseVector(new double[] {-2.0, -2.0}); + assertEquals(originalMdl.apply(firstVector), updatedMdlOnSameDataset.apply(firstVector), PRECISION); + assertEquals(originalMdl.apply(secondVector), updatedMdlOnSameDataset.apply(secondVector), PRECISION); + assertEquals(originalMdl.apply(firstVector), updatedMdlOnEmptyDataset.apply(firstVector), PRECISION); + assertEquals(originalMdl.apply(secondVector), updatedMdlOnEmptyDataset.apply(secondVector), PRECISION); + } + + /** */ + @NotNull private KMeansTrainer createAndCheckTrainer() { KMeansTrainer trainer = new KMeansTrainer() .withDistance(new EuclideanDistance()) .withK(10) @@ -61,20 +113,6 @@ public void findOneClusters() { assertEquals(10, trainer.getK()); assertEquals(2, trainer.getSeed()); assertTrue(trainer.getDistance() instanceof EuclideanDistance); - - KMeansModel knnMdl = trainer - .withK(1) - .fit( - new LocalDatasetBuilder<>(data, 2), - (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), - (k, v) -> v[2] - ); - - Vector firstVector = new DenseVector(new double[]{2.0, 2.0}); - assertEquals(knnMdl.apply(firstVector), 0.0, PRECISION); - Vector secondVector = new DenseVector(new double[]{-2.0, -2.0}); - assertEquals(knnMdl.apply(secondVector), 0.0, PRECISION); - assertEquals(trainer.getMaxIterations(), 1); - assertEquals(trainer.getEpsilon(), PRECISION, PRECISION); + return trainer; } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/common/CollectionsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/common/CollectionsTest.java index acf28e96014fc..745eac946f4f7 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/common/CollectionsTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/common/CollectionsTest.java @@ -22,6 +22,7 @@ import org.apache.ignite.ml.clustering.kmeans.KMeansModel; import org.apache.ignite.ml.clustering.kmeans.KMeansModelFormat; import org.apache.ignite.ml.knn.ann.ANNClassificationModel; +import org.apache.ignite.ml.knn.ann.ANNClassificationTrainer; import org.apache.ignite.ml.knn.ann.ANNModelFormat; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; import org.apache.ignite.ml.knn.classification.KNNModelFormat; @@ -103,11 +104,11 @@ public void test() { test(new SVMLinearBinaryClassificationModel(null, 1.0), new SVMLinearBinaryClassificationModel(null, 0.5)); - test(new ANNClassificationModel(new LabeledVectorSet<>()), - new ANNClassificationModel(new LabeledVectorSet<>(1, 1, true))); + test(new ANNClassificationModel(new LabeledVectorSet<>(), new ANNClassificationTrainer.CentroidStat()), + new ANNClassificationModel(new LabeledVectorSet<>(1, 1, true), new ANNClassificationTrainer.CentroidStat())); - test(new ANNModelFormat(1, new ManhattanDistance(), NNStrategy.SIMPLE, new LabeledVectorSet<>()), - new ANNModelFormat(2, new ManhattanDistance(), NNStrategy.SIMPLE, new LabeledVectorSet<>())); + test(new ANNModelFormat(1, new ManhattanDistance(), NNStrategy.SIMPLE, new LabeledVectorSet<>(), new ANNClassificationTrainer.CentroidStat()), + new ANNModelFormat(2, new ManhattanDistance(), NNStrategy.SIMPLE, new LabeledVectorSet<>(), new ANNClassificationTrainer.CentroidStat())); } /** Test classes that have all instances equal (eg, metrics). */ diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java index 17d9c1a3a03cb..9315850ace7bf 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.knn.NNClassificationModel; import org.apache.ignite.ml.knn.ann.ANNClassificationModel; +import org.apache.ignite.ml.knn.ann.ANNClassificationTrainer; import org.apache.ignite.ml.knn.ann.ANNModelFormat; import org.apache.ignite.ml.knn.ann.ProbableLabel; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; @@ -237,7 +238,7 @@ public void importExportANNModelTest() throws IOException { executeModelTest(mdlFilePath -> { final LabeledVectorSet centers = new LabeledVectorSet<>(); - NNClassificationModel mdl = new ANNClassificationModel(centers) + NNClassificationModel mdl = new ANNClassificationModel(centers, new ANNClassificationTrainer.CentroidStat()) .withK(4) .withDistanceMeasure(new ManhattanDistance()) .withStrategy(NNStrategy.WEIGHTED); @@ -250,7 +251,7 @@ public void importExportANNModelTest() throws IOException { Assert.assertNotNull(load); - NNClassificationModel importedMdl = new ANNClassificationModel(load.getCandidates()) + NNClassificationModel importedMdl = new ANNClassificationModel(load.getCandidates(), new ANNClassificationTrainer.CentroidStat()) .withK(load.getK()) .withDistanceMeasure(load.getDistanceMeasure()) .withStrategy(load.getStgy()); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java index 4452668207bf5..3e340f6047600 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java @@ -54,7 +54,7 @@ public void testFitRegression() { learningSample.put(i, new double[] {xs[i], ys[i]}); } - DatasetTrainer, Double> trainer + DatasetTrainer trainer = new GDBRegressionOnTreesTrainer(1.0, 2000, 3, 0.0).withUseIndex(true); Model mdl = trainer.fit( diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java index 7289b1dfaf1f7..d8fb620d16663 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java @@ -26,6 +26,7 @@ import org.apache.ignite.ml.knn.ann.ANNClassificationTrainer; import org.apache.ignite.ml.knn.classification.NNStrategy; import org.apache.ignite.ml.math.distances.EuclideanDistance; +import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.junit.Assert; import org.junit.Test; @@ -68,4 +69,47 @@ public void testBinaryClassification() { Assert.assertTrue(mdl.toString(true).contains(NNStrategy.SIMPLE.name())); Assert.assertTrue(mdl.toString(false).contains(NNStrategy.SIMPLE.name())); } + + /** */ + @Test + public void testUpdate() { + Map cacheMock = new HashMap<>(); + + for (int i = 0; i < twoClusters.length; i++) + cacheMock.put(i, twoClusters[i]); + + ANNClassificationTrainer trainer = new ANNClassificationTrainer() + .withK(10) + .withMaxIterations(10) + .withEpsilon(1e-4) + .withDistance(new EuclideanDistance()); + + ANNClassificationModel originalMdl = (ANNClassificationModel) trainer.withSeed(1234L).fit( + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ).withK(3) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(NNStrategy.SIMPLE); + + ANNClassificationModel updatedOnSameDataset = trainer.withSeed(1234L).update(originalMdl, + cacheMock, parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + ANNClassificationModel updatedOnEmptyDataset = trainer.withSeed(1234L).update(originalMdl, + new HashMap(), parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + Vector v1 = VectorUtils.of(550, 550); + Vector v2 = VectorUtils.of(-550, -550); + TestUtils.assertEquals(originalMdl.apply(v1), updatedOnSameDataset.apply(v1), PRECISION); + TestUtils.assertEquals(originalMdl.apply(v2), updatedOnSameDataset.apply(v2), PRECISION); + TestUtils.assertEquals(originalMdl.apply(v1), updatedOnEmptyDataset.apply(v1), PRECISION); + TestUtils.assertEquals(originalMdl.apply(v2), updatedOnEmptyDataset.apply(v2), PRECISION); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java index c5a5c1cb2aeee..748123a18d0ca 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java @@ -174,4 +174,43 @@ public void testBinaryClassificationFarPointsWithWeightedStrategy() { Vector vector = new DenseVector(new double[] {-1.01, -1.01}); assertEquals(knnMdl.apply(vector), 1.0); } + + /** */ + @Test + public void testUpdate() { + Map data = new HashMap<>(); + data.put(0, new double[] {10.0, 10.0, 1.0}); + data.put(1, new double[] {10.0, 20.0, 1.0}); + data.put(2, new double[] {-1, -1, 1.0}); + data.put(3, new double[] {-2, -2, 2.0}); + data.put(4, new double[] {-1.0, -2.0, 2.0}); + data.put(5, new double[] {-2.0, -1.0, 2.0}); + + KNNClassificationTrainer trainer = new KNNClassificationTrainer(); + + KNNClassificationModel originalMdl = (KNNClassificationModel)trainer.fit( + data, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ).withK(3) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(NNStrategy.WEIGHTED); + + KNNClassificationModel updatedOnSameDataset = trainer.update(originalMdl, + data, parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + KNNClassificationModel updatedOnEmptyDataset = trainer.update(originalMdl, + new HashMap(), parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + Vector vector = new DenseVector(new double[] {-1.01, -1.01}); + assertEquals(originalMdl.apply(vector), updatedOnSameDataset.apply(vector)); + assertEquals(originalMdl.apply(vector), updatedOnEmptyDataset.apply(vector)); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java index 5504e1ac63feb..52ff1ecd9a7d5 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java @@ -35,6 +35,8 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import static junit.framework.TestCase.assertEquals; + /** * Tests for {@link KNNRegressionTrainer}. */ @@ -135,4 +137,42 @@ private void testLongly(NNStrategy stgy) { Assert.assertTrue(knnMdl.toString(true).contains(stgy.name())); Assert.assertTrue(knnMdl.toString(false).contains(stgy.name())); } + + /** */ + @Test + public void testUpdate() { + Map data = new HashMap<>(); + data.put(0, new double[] {11.0, 0, 0, 0, 0, 0}); + data.put(1, new double[] {12.0, 2.0, 0, 0, 0, 0}); + data.put(2, new double[] {13.0, 0, 3.0, 0, 0, 0}); + data.put(3, new double[] {14.0, 0, 0, 4.0, 0, 0}); + data.put(4, new double[] {15.0, 0, 0, 0, 5.0, 0}); + data.put(5, new double[] {16.0, 0, 0, 0, 0, 6.0}); + + KNNRegressionTrainer trainer = new KNNRegressionTrainer(); + + KNNRegressionModel originalMdl = (KNNRegressionModel) trainer.fit( + new LocalDatasetBuilder<>(data, parts), + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ).withK(1) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(NNStrategy.SIMPLE); + + KNNRegressionModel updatedOnSameDataset = trainer.update(originalMdl, + data, parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + KNNRegressionModel updatedOnEmptyDataset = trainer.update(originalMdl, + new HashMap(), parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[2] + ); + + Vector vector = new DenseVector(new double[] {0, 0, 0, 5.0, 0.0}); + assertEquals(originalMdl.apply(vector), updatedOnSameDataset.apply(vector)); + assertEquals(originalMdl.apply(vector), updatedOnEmptyDataset.apply(vector)); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java index a1d601cbb7428..6a6555e7e8125 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java @@ -29,6 +29,7 @@ import org.apache.ignite.ml.math.primitives.vector.impl.DenseVector; import org.apache.ignite.ml.nn.architecture.MLPArchitecture; import org.apache.ignite.ml.optimization.LossFunctions; +import org.apache.ignite.ml.optimization.SmoothParametrized; import org.apache.ignite.ml.optimization.updatecalculators.NesterovParameterUpdate; import org.apache.ignite.ml.optimization.updatecalculators.NesterovUpdateCalculator; import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; @@ -154,6 +155,69 @@ private

      void xorTest(UpdatesStrategy updatesStgy = new UpdatesStrategy<>( + new SimpleGDUpdateCalculator(0.2), + SimpleGDParameterUpdate::sumLocal, + SimpleGDParameterUpdate::avg + ); + + Map xorData = new HashMap<>(); + xorData.put(0, new double[][]{{0.0, 0.0}, {0.0}}); + xorData.put(1, new double[][]{{0.0, 1.0}, {1.0}}); + xorData.put(2, new double[][]{{1.0, 0.0}, {1.0}}); + xorData.put(3, new double[][]{{1.0, 1.0}, {0.0}}); + + MLPArchitecture arch = new MLPArchitecture(2). + withAddedLayer(10, true, Activators.RELU). + withAddedLayer(1, false, Activators.SIGMOID); + + MLPTrainer trainer = new MLPTrainer<>( + arch, + LossFunctions.MSE, + updatesStgy, + 3000, + batchSize, + 50, + 123L + ); + + MultilayerPerceptron originalMdl = trainer.fit( + xorData, + parts, + (k, v) -> VectorUtils.of(v[0]), + (k, v) -> v[1] + ); + + MultilayerPerceptron updatedOnSameDS = trainer.update( + originalMdl, + xorData, + parts, + (k, v) -> VectorUtils.of(v[0]), + (k, v) -> v[1] + ); + + MultilayerPerceptron updatedOnEmptyDS = trainer.update( + originalMdl, + new HashMap(), + parts, + (k, v) -> VectorUtils.of(v[0]), + (k, v) -> v[1] + ); + + DenseMatrix matrix = new DenseMatrix(new double[][] { + {0.0, 0.0}, + {0.0, 1.0}, + {1.0, 0.0}, + {1.0, 1.0} + }); + + TestUtils.checkIsInEpsilonNeighbourhood(originalMdl.apply(matrix).getRow(0), updatedOnSameDS.apply(matrix).getRow(0), 1E-1); + TestUtils.checkIsInEpsilonNeighbourhood(originalMdl.apply(matrix).getRow(0), updatedOnEmptyDS.apply(matrix).getRow(0), 1E-1); + } } /** diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java index d16ae72e15dde..9c35ac7f7394e 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java @@ -101,4 +101,55 @@ public void testBigDataFit() { assertEquals(intercept, mdl.getIntercept(), 1e-6); } + + /** */ + @Test + public void testUpdate() { + Random rnd = new Random(0); + Map data = new HashMap<>(); + double[] coef = new double[100]; + double intercept = rnd.nextDouble() * 10; + + for (int i = 0; i < 100000; i++) { + double[] x = new double[coef.length + 1]; + + for (int j = 0; j < coef.length; j++) + x[j] = rnd.nextDouble() * 10; + + x[coef.length] = intercept; + + data.put(i, x); + } + + LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer(); + + LinearRegressionModel originalModel = trainer.fit( + data, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[coef.length] + ); + + LinearRegressionModel updatedOnSameDS = trainer.update( + originalModel, + data, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[coef.length] + ); + + LinearRegressionModel updatedOnEmpyDS = trainer.update( + originalModel, + new HashMap(), + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[coef.length] + ); + + assertArrayEquals(originalModel.getWeights().getStorage().data(), updatedOnSameDS.getWeights().getStorage().data(), 1e-6); + assertEquals(originalModel.getIntercept(), updatedOnSameDS.getIntercept(), 1e-6); + + assertArrayEquals(originalModel.getWeights().getStorage().data(), updatedOnEmpyDS.getWeights().getStorage().data(), 1e-6); + assertEquals(originalModel.getIntercept(), updatedOnEmpyDS.getIntercept(), 1e-6); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java index 349e712620e49..86b0f275d3208 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java @@ -72,4 +72,66 @@ public void testSmallDataFit() { assertEquals(2.8421709430404007e-14, mdl.getIntercept(), 1e-1); } + + /** */ + @Test + public void testUpdate() { + Map data = new HashMap<>(); + data.put(0, new double[]{-1.0915526, 1.81983527, -0.91409478, 0.70890712, -24.55724107}); + data.put(1, new double[]{-0.61072904, 0.37545517, 0.21705352, 0.09516495, -26.57226867}); + data.put(2, new double[]{0.05485406, 0.88219898, -0.80584547, 0.94668307, 61.80919728}); + data.put(3, new double[]{-0.24835094, -0.34000053, -1.69984651, -1.45902635, -161.65525991}); + data.put(4, new double[]{0.63675392, 0.31675535, 0.38837437, -1.1221971, -14.46432611}); + data.put(5, new double[]{0.14194017, 2.18158997, -0.28397346, -0.62090588, -3.2122197}); + data.put(6, new double[]{-0.53487507, 1.4454797, 0.21570443, -0.54161422, -46.5469012}); + data.put(7, new double[]{-1.58812173, -0.73216803, -2.15670676, -1.03195988, -247.23559889}); + data.put(8, new double[]{0.20702671, 0.92864654, 0.32721202, -0.09047503, 31.61484949}); + data.put(9, new double[]{-0.37890345, -0.04846179, -0.84122753, -1.14667474, -124.92598583}); + + LinearRegressionSGDTrainer trainer = new LinearRegressionSGDTrainer<>(new UpdatesStrategy<>( + new RPropUpdateCalculator(), + RPropParameterUpdate::sumLocal, + RPropParameterUpdate::avg + ), 100000, 10, 100, 0L); + + LinearRegressionModel originalModel = trainer.withSeed(0).fit( + data, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[4] + ); + + + LinearRegressionModel updatedOnSameDS = trainer.withSeed(0).update( + originalModel, + data, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[4] + ); + + LinearRegressionModel updatedOnEmptyDS = trainer.withSeed(0).update( + originalModel, + new HashMap(), + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), + (k, v) -> v[4] + ); + + assertArrayEquals( + originalModel.getWeights().getStorage().data(), + updatedOnSameDS.getWeights().getStorage().data(), + 1.0 + ); + + assertEquals(originalModel.getIntercept(), updatedOnSameDS.getIntercept(), 1.0); + + assertArrayEquals( + originalModel.getWeights().getStorage().data(), + updatedOnEmptyDS.getWeights().getStorage().data(), + 1e-1 + ); + + assertEquals(originalModel.getIntercept(), updatedOnEmptyDS.getIntercept(), 1e-1); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java index 1f8c5d12670c4..f08501ca468a3 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java @@ -19,9 +19,11 @@ import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.ignite.ml.TestUtils; import org.apache.ignite.ml.common.TrainerTest; +import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.nn.UpdatesStrategy; import org.apache.ignite.ml.optimization.SmoothParametrized; @@ -81,4 +83,60 @@ public void testTrainWithTheLinearlySeparableCase() { TestUtils.assertEquals(2, mdl.apply(VectorUtils.of(-10, -10)), PRECISION); TestUtils.assertEquals(3, mdl.apply(VectorUtils.of(10, -10)), PRECISION); } + + /** */ + @Test + public void testUpdate() { + Map cacheMock = new HashMap<>(); + + for (int i = 0; i < fourSetsInSquareVertices.length; i++) + cacheMock.put(i, fourSetsInSquareVertices[i]); + + LogRegressionMultiClassTrainer trainer = new LogRegressionMultiClassTrainer<>() + .withUpdatesStgy(new UpdatesStrategy<>( + new SimpleGDUpdateCalculator(0.2), + SimpleGDParameterUpdate::sumLocal, + SimpleGDParameterUpdate::avg + )) + .withAmountOfIterations(1000) + .withAmountOfLocIterations(10) + .withBatchSize(100) + .withSeed(123L); + + LogRegressionMultiClassModel originalModel = trainer.fit( + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + LogRegressionMultiClassModel updatedOnSameDS = trainer.update( + originalModel, + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + LogRegressionMultiClassModel updatedOnEmptyDS = trainer.update( + originalModel, + new HashMap(), + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + List vectors = Arrays.asList( + VectorUtils.of(10, 10), + VectorUtils.of(-10, 10), + VectorUtils.of(-10, -10), + VectorUtils.of(10, -10) + ); + + + for (Vector vec : vectors) { + TestUtils.assertEquals(originalModel.apply(vec), updatedOnSameDS.apply(vec), PRECISION); + TestUtils.assertEquals(originalModel.apply(vec), updatedOnEmptyDS.apply(vec), PRECISION); + } + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogisticRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogisticRegressionSGDTrainerTest.java index 5bd2dbd9811e2..1da0d1ab09273 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogisticRegressionSGDTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogisticRegressionSGDTrainerTest.java @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.ignite.ml.TestUtils; import org.apache.ignite.ml.common.TrainerTest; +import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.nn.UpdatesStrategy; import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate; @@ -60,4 +61,49 @@ public void trainWithTheLinearlySeparableCase() { TestUtils.assertEquals(0, mdl.apply(VectorUtils.of(100, 10)), PRECISION); TestUtils.assertEquals(1, mdl.apply(VectorUtils.of(10, 100)), PRECISION); } + + /** */ + @Test + public void testUpdate() { + Map cacheMock = new HashMap<>(); + + for (int i = 0; i < twoLinearlySeparableClasses.length; i++) + cacheMock.put(i, twoLinearlySeparableClasses[i]); + + LogisticRegressionSGDTrainer trainer = new LogisticRegressionSGDTrainer<>(new UpdatesStrategy<>( + new SimpleGDUpdateCalculator().withLearningRate(0.2), + SimpleGDParameterUpdate::sumLocal, + SimpleGDParameterUpdate::avg + ), 100000, 10, 100, 123L); + + LogisticRegressionModel originalModel = trainer.fit( + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + LogisticRegressionModel updatedOnSameDS = trainer.update( + originalModel, + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + LogisticRegressionModel updatedOnEmptyDS = trainer.update( + originalModel, + new HashMap(), + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + Vector v1 = VectorUtils.of(100, 10); + Vector v2 = VectorUtils.of(10, 100); + TestUtils.assertEquals(originalModel.apply(v1), updatedOnSameDS.apply(v1), PRECISION); + TestUtils.assertEquals(originalModel.apply(v2), updatedOnSameDS.apply(v2), PRECISION); + TestUtils.assertEquals(originalModel.apply(v2), updatedOnEmptyDS.apply(v2), PRECISION); + TestUtils.assertEquals(originalModel.apply(v1), updatedOnEmptyDS.apply(v1), PRECISION); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java index 5630beea5f370..263bb6d65fb71 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.ignite.ml.TestUtils; import org.apache.ignite.ml.common.TrainerTest; +import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.junit.Test; @@ -52,4 +53,44 @@ public void testTrainWithTheLinearlySeparableCase() { TestUtils.assertEquals(-1, mdl.apply(VectorUtils.of(100, 10)), PRECISION); TestUtils.assertEquals(1, mdl.apply(VectorUtils.of(10, 100)), PRECISION); } + + /** */ + @Test + public void testUpdate() { + Map cacheMock = new HashMap<>(); + + for (int i = 0; i < twoLinearlySeparableClasses.length; i++) + cacheMock.put(i, twoLinearlySeparableClasses[i]); + + SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer() + .withAmountOfIterations(1000) + .withSeed(1234L); + + SVMLinearBinaryClassificationModel originalModel = trainer.fit( + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + SVMLinearBinaryClassificationModel updatedOnSameDS = trainer.update( + originalModel, + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + SVMLinearBinaryClassificationModel updatedOnEmptyDS = trainer.update( + originalModel, + new HashMap(), + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + Vector v = VectorUtils.of(100, 10); + TestUtils.assertEquals(originalModel.apply(v), updatedOnSameDS.apply(v), PRECISION); + TestUtils.assertEquals(originalModel.apply(v), updatedOnEmptyDS.apply(v), PRECISION); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java index 7ea28c2493f26..e0c62af2cc21c 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.ignite.ml.TestUtils; import org.apache.ignite.ml.common.TrainerTest; +import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.junit.Test; @@ -54,4 +55,46 @@ public void testTrainWithTheLinearlySeparableCase() { TestUtils.assertEquals(0, mdl.apply(VectorUtils.of(100, 10)), PRECISION); TestUtils.assertEquals(1, mdl.apply(VectorUtils.of(10, 100)), PRECISION); } + + /** */ + @Test + public void testUpdate() { + Map cacheMock = new HashMap<>(); + + for (int i = 0; i < twoLinearlySeparableClasses.length; i++) + cacheMock.put(i, twoLinearlySeparableClasses[i]); + + SVMLinearMultiClassClassificationTrainer trainer = new SVMLinearMultiClassClassificationTrainer() + .withLambda(0.3) + .withAmountOfLocIterations(10) + .withAmountOfIterations(100) + .withSeed(1234L); + + SVMLinearMultiClassClassificationModel originalModel = trainer.fit( + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + SVMLinearMultiClassClassificationModel updatedOnSameDS = trainer.update( + originalModel, + cacheMock, + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + SVMLinearMultiClassClassificationModel updatedOnEmptyDS = trainer.update( + originalModel, + new HashMap(), + parts, + (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)), + (k, v) -> v[0] + ); + + Vector v = VectorUtils.of(100, 10); + TestUtils.assertEquals(originalModel.apply(v), updatedOnSameDS.apply(v), PRECISION); + TestUtils.assertEquals(originalModel.apply(v), updatedOnEmptyDS.apply(v), PRECISION); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java index 4abf5083170cf..087f4e827bd63 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestClassifierTrainerTest.java @@ -24,6 +24,7 @@ import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.composition.predictionsaggregator.OnMajorityPredictionsAggregator; import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.junit.Test; import org.junit.runner.RunWith; @@ -75,7 +76,7 @@ public void testFit() { } ArrayList meta = new ArrayList<>(); - for(int i = 0; i < 4; i++) + for (int i = 0; i < 4; i++) meta.add(new FeatureMeta("", i, false)); RandomForestClassifierTrainer trainer = new RandomForestClassifierTrainer(meta) .withCountOfTrees(5) @@ -86,4 +87,34 @@ public void testFit() { assertTrue(mdl.getPredictionsAggregator() instanceof OnMajorityPredictionsAggregator); assertEquals(5, mdl.getModels().size()); } + + /** */ + @Test + public void testUpdate() { + int sampleSize = 1000; + Map sample = new HashMap<>(); + for (int i = 0; i < sampleSize; i++) { + double x1 = i; + double x2 = x1 / 10.0; + double x3 = x2 / 10.0; + double x4 = x3 / 10.0; + + sample.put(new double[] {x1, x2, x3, x4}, (double)(i % 2)); + } + + ArrayList meta = new ArrayList<>(); + for (int i = 0; i < 4; i++) + meta.add(new FeatureMeta("", i, false)); + RandomForestClassifierTrainer trainer = new RandomForestClassifierTrainer(meta) + .withCountOfTrees(100) + .withFeaturesCountSelectionStrgy(x -> 2); + + ModelsComposition originalModel = trainer.fit(sample, parts, (k, v) -> VectorUtils.of(k), (k, v) -> v); + ModelsComposition updatedOnSameDS = trainer.update(originalModel, sample, parts, (k, v) -> VectorUtils.of(k), (k, v) -> v); + ModelsComposition updatedOnEmptyDS = trainer.update(originalModel, new HashMap(), parts, (k, v) -> VectorUtils.of(k), (k, v) -> v); + + Vector v = VectorUtils.of(5, 0.5, 0.05, 0.005); + assertEquals(originalModel.apply(v), updatedOnSameDS.apply(v), 0.01); + assertEquals(originalModel.apply(v), updatedOnEmptyDS.apply(v), 0.01); + } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java index c4a4a750db604..fcc20bd9b4cb3 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestRegressionTrainerTest.java @@ -24,6 +24,7 @@ import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.composition.predictionsaggregator.MeanValuePredictionsAggregator; import org.apache.ignite.ml.dataset.feature.FeatureMeta; +import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.junit.Test; import org.junit.runner.RunWith; @@ -82,4 +83,34 @@ public void testFit() { assertTrue(mdl.getPredictionsAggregator() instanceof MeanValuePredictionsAggregator); assertEquals(5, mdl.getModels().size()); } + + /** */ + @Test + public void testUpdate() { + int sampleSize = 1000; + Map sample = new HashMap<>(); + for (int i = 0; i < sampleSize; i++) { + double x1 = i; + double x2 = x1 / 10.0; + double x3 = x2 / 10.0; + double x4 = x3 / 10.0; + + sample.put(new double[] {x1, x2, x3, x4}, (double)(i % 2)); + } + + ArrayList meta = new ArrayList<>(); + for (int i = 0; i < 4; i++) + meta.add(new FeatureMeta("", i, false)); + RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer(meta) + .withCountOfTrees(100) + .withFeaturesCountSelectionStrgy(x -> 2); + + ModelsComposition originalModel = trainer.fit(sample, parts, (k, v) -> VectorUtils.of(k), (k, v) -> v); + ModelsComposition updatedOnSameDS = trainer.update(originalModel, sample, parts, (k, v) -> VectorUtils.of(k), (k, v) -> v); + ModelsComposition updatedOnEmptyDS = trainer.update(originalModel, new HashMap(), parts, (k, v) -> VectorUtils.of(k), (k, v) -> v); + + Vector v = VectorUtils.of(5, 0.5, 0.05, 0.005); + assertEquals(originalModel.apply(v), updatedOnSameDS.apply(v), 0.1); + assertEquals(originalModel.apply(v), updatedOnEmptyDS.apply(v), 0.1); + } } From cdfed61dcb8599b9670e1373e09af7b103c7ff32 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Tue, 4 Sep 2018 17:26:58 +0300 Subject: [PATCH 46/95] IGNITE-9054 Avoid using OptimizedMarshaller with initial ScanQuery. - Fixes #4592. Signed-off-by: Alexey Goncharuk --- .../cache/query/GridCacheQueryManager.java | 4 +- .../query/GridCacheQueryResponseEntry.java | 5 +- .../custom/DummyEventFilterFactory.java | 10 +- .../ContinuousQueryMarshallerTest.java | 168 ++++++++++++++++++ .../ContinuousQueryPeerClassLoadingTest.java | 4 +- .../IgniteCacheQuerySelfTestSuite3.java | 2 + 6 files changed, 183 insertions(+), 10 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryMarshallerTest.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 281400e30e620..982006f6c3807 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 @@ -1278,7 +1278,7 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { continue; } else - data.add(!loc ? new GridCacheQueryResponseEntry<>(key, val) : F.t(key, val)); + data.add(new T2<>(key, val)); } if (!loc) { @@ -3119,7 +3119,7 @@ private void advance() { } } else - next0 = !locNode ? new GridCacheQueryResponseEntry<>(key0, val0): + next0 = !locNode ? new T2<>(key0, val0): new CacheQueryEntry<>(key0, val0); break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponseEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponseEntry.java index 2c1a4f51aa0d7..650f0c03d1f49 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponseEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponseEntry.java @@ -27,7 +27,10 @@ /** * Class to store query results returned by remote nodes. It's required to fully * control serialization process. Local entries can be returned to user as is. + *

      + * @deprecated Should be removed in Apache Ignite 3.0. */ +@Deprecated public class GridCacheQueryResponseEntry implements Map.Entry, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -113,4 +116,4 @@ public GridCacheQueryResponseEntry(K key, V val) { @Override public String toString() { return "[" + key + "=" + val + "]"; } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/custom/DummyEventFilterFactory.java b/modules/core/src/test/java/org/apache/ignite/custom/DummyEventFilterFactory.java index e0688bccd934d..103e6a8a0a409 100644 --- a/modules/core/src/test/java/org/apache/ignite/custom/DummyEventFilterFactory.java +++ b/modules/core/src/test/java/org/apache/ignite/custom/DummyEventFilterFactory.java @@ -25,22 +25,22 @@ /** * Must be not in org.apache.ignite.internal */ -public class DummyEventFilterFactory implements Factory> { +public class DummyEventFilterFactory implements Factory> { /** */ private static final long serialVersionUID = 0L; /** {@inheritDoc} */ - @Override public CacheEntryEventFilter create() { - return new DummyEventFilter(); + @Override public CacheEntryEventFilter create() { + return new DummyEventFilter(); } /** * */ - private static class DummyEventFilter implements CacheEntryEventFilter { + private static class DummyEventFilter implements CacheEntryEventFilter { /** {@inheritDoc} */ @Override public boolean evaluate( - final CacheEntryEvent evt) throws CacheEntryListenerException { + final CacheEntryEvent evt) throws CacheEntryListenerException { return true; } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryMarshallerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryMarshallerTest.java new file mode 100644 index 0000000000000..44dcc1cd63f4c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryMarshallerTest.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.processors.cache.query.continuous; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.ObjectStreamException; +import java.io.Serializable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import javax.cache.Cache; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryListenerException; +import javax.cache.event.CacheEntryUpdatedListener; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; +import org.apache.ignite.cache.query.ContinuousQuery; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.custom.DummyEventFilterFactory; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Checks that Optimized Marshaller is not used on any stage of Continuous Query handling. + */ +public class ContinuousQueryMarshallerTest extends GridCommonAbstractTest { + /** */ + public static final String CACHE_NAME = "test-cache"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(gridName.contains("client")); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testRemoteFilterFactoryClient() throws Exception { + check("server", "client"); + } + + /** + * @throws Exception If failed. + */ + public void testRemoteFilterFactoryServer() throws Exception { + check("server1", "server2"); + } + + /** + * @param node1Name Node 1 name. + * @param node2Name Node 2 name. + */ + private void check(String node1Name, String node2Name) throws Exception { + final Ignite node1 = startGrid(node1Name); + + final IgniteCache cache = node1.getOrCreateCache(CACHE_NAME); + + for (int i = 0; i < 10; i++) + cache.put(i, new MarshallerCheckingEntry(String.valueOf(i))); + + final Ignite node2 = startGrid(node2Name); + + final ContinuousQuery qry = new ContinuousQuery<>(); + + ScanQuery scanQry = new ScanQuery<>(new IgniteBiPredicate() { + @Override public boolean apply(Integer key, MarshallerCheckingEntry val) { + return key % 2 == 0; + } + }); + + qry.setInitialQuery(scanQry); + + qry.setRemoteFilterFactory(new DummyEventFilterFactory<>()); + + final CountDownLatch latch = new CountDownLatch(15); + + qry.setLocalListener(new CacheEntryUpdatedListener() { + @Override public void onUpdated( + final Iterable> evts) + throws CacheEntryListenerException { + + System.out.println(">> Client 1 events " + evts); + + for (CacheEntryEvent evt : evts) + latch.countDown(); + } + }); + + final IgniteCache cache1 = node2.cache(CACHE_NAME); + + for (Cache.Entry entry : cache1.query(qry)) { + latch.countDown(); + System.out.println(">> Initial entry " + entry); + } + + for (int i = 10; i < 20; i++) + cache1.put(i, new MarshallerCheckingEntry(i)); + + assertTrue(Long.toString(latch.getCount()), latch.await(5, TimeUnit.SECONDS)); + } + + /** Checks that OptimizedMarshaller is never used (BinaryMarshaller is OK) */ + private class MarshallerCheckingEntry implements Serializable, Binarylizable { + /** */ + private Object val; + + /** */ + public MarshallerCheckingEntry(Object val) { + this.val = val; + } + + /** */ + private void writeObject(ObjectOutputStream out) throws IOException { + throw new UnsupportedOperationException(); + } + + /** */ + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + throw new UnsupportedOperationException(); + } + + /** */ + private void readObjectNoData() throws ObjectStreamException { + throw new UnsupportedOperationException(); + } + + /** */ + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + writer.writeObject("value", val); + } + + /** */ + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + val = reader.readObject("value"); + } + } + +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryPeerClassLoadingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryPeerClassLoadingTest.java index e5d1d609d7052..73d8d0d99fe76 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryPeerClassLoadingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryPeerClassLoadingTest.java @@ -91,8 +91,8 @@ private void check(String node1Name, String node2Name, String node3Name) throws final ContinuousQuery qry1 = new ContinuousQuery<>(); final ContinuousQuery qry2 = new ContinuousQuery<>(); - qry1.setRemoteFilterFactory(new DummyEventFilterFactory()); - qry2.setRemoteFilterFactory(new DummyEventFilterFactory()); + qry1.setRemoteFilterFactory(new DummyEventFilterFactory<>()); + qry2.setRemoteFilterFactory(new DummyEventFilterFactory<>()); final AtomicInteger client1Evts = new AtomicInteger(0); final AtomicInteger client2Evts = new AtomicInteger(0); 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 e810d300ea990..08511d919d20f 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 @@ -47,6 +47,7 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationStoreEnabledTest; import org.apache.ignite.internal.processors.cache.query.continuous.ClientReconnectContinuousQueryTest; +import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryMarshallerTest; import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryPeerClassLoadingTest; import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryRemoteFilterMissingInClassPathSelfTest; import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest; @@ -132,6 +133,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(ClientReconnectContinuousQueryTest.class); suite.addTestSuite(ContinuousQueryPeerClassLoadingTest.class); suite.addTestSuite(ClientReconnectContinuousQueryTest.class); + suite.addTestSuite(ContinuousQueryMarshallerTest.class); suite.addTestSuite(CacheContinuousQueryConcurrentPartitionUpdateTest.class); suite.addTestSuite(CacheContinuousQueryEventBufferTest.class); From f15dfb0f462ca1c5d8fc566f4906ab7d8c22da7c Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Tue, 4 Sep 2018 18:49:06 +0300 Subject: [PATCH 47/95] IGNITE-9360 Remove SnapTreeMap and tests on it. - Fixes #4608. Signed-off-by: Dmitriy Govorukhin --- .../util/snaptree/CopyOnWriteManager.java | 390 --- .../ignite/internal/util/snaptree/Epoch.java | 131 - .../internal/util/snaptree/EpochNode.java | 433 --- .../internal/util/snaptree/SnapTreeMap.java | 2917 ----------------- .../internal/util/snaptree/package-info.java | 22 - .../testsuites/IgniteUtilSelfTestSuite.java | 2 - .../apache/ignite/util/GridIndexFillTest.java | 259 -- .../loadtests/h2indexing/GridTestEntity.java | 67 - .../h2indexing/GridTreeBenchmark.java | 280 -- 9 files changed, 4501 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/CopyOnWriteManager.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/Epoch.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/EpochNode.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/SnapTreeMap.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/package-info.java delete mode 100644 modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTestEntity.java delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTreeBenchmark.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/CopyOnWriteManager.java b/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/CopyOnWriteManager.java deleted file mode 100644 index 685668a89b98f..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/CopyOnWriteManager.java +++ /dev/null @@ -1,390 +0,0 @@ -/* - * Copyright (c) 2009 Stanford University, unless otherwise specified. - * All rights reserved. - * - * This software was developed by the Pervasive Parallelism Laboratory of - * Stanford University, California, USA. - * - * Permission to use, copy, modify, and distribute this software in source - * or binary form for any purpose with or without fee is hereby granted, - * provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * - * 3. Neither the name of Stanford University nor the names of its - * contributors may be used to endorse or promote products derived - * from this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND - * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY - * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF - * SUCH DAMAGE. - */ - -package org.apache.ignite.internal.util.snaptree; - -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.AbstractQueuedSynchronizer; - -/** Manages copy-on-write behavior for a concurrent tree structure. It is - * assumed that the managed structure allows concurrent mutation, but that no - * mutating operations may be active when a copy-on-write snapshot of tree is - * taken. Because it is difficult to update the size of data structure in a - * highly concurrent fashion, the CopyOnWriteManager also manages - * a running total that represents the size of the contained tree structure. - *

      - * Users should implement the {@link #freezeAndClone(Object)} and - * {@link #cloneFrozen(Object)} methods. - */ -@SuppressWarnings("ALL") -abstract public class CopyOnWriteManager implements Cloneable { - - /** This is basically a stripped-down CountDownLatch. Implementing our own - * reduces the object count by one, and it gives us access to the - * uninterruptable acquireShared. - */ - private class Latch extends AbstractQueuedSynchronizer { - /** */ - private static final long serialVersionUID = 0L; - - Latch(final boolean triggered) { - setState(triggered ? 0 : 1); - } - - public int tryAcquireShared(final int acquires) { - // 1 = success, and followers may also succeed - // -1 = failure - return getState() == 0 ? 1 : -1; - } - - public boolean tryReleaseShared(final int releases) { - // Before, state is either 0 or 1. After, state is always 0. - return compareAndSetState(1, 0); - } - } - - private static final int MUTATE = 1; - private static final int MUTATE_AFTER_FREEZE = 2; - private static final int BULK_READ = 3; - private static final int BULK_READ_AFTER_FREEZE = 4; - - private class COWEpoch extends EpochNode { - /** */ - private static final long serialVersionUID = 0L; - - /** Tripped after this COWEpoch is installed as active. */ - private final Latch _activated; - - /** True iff this is a mutating epoch. */ - final boolean mutationAllowed; - - /** The value used by this epoch. */ - E value; - - /** The computed size of value, as of the beginning of - * this epoch. - */ - int initialSize; - - /** A frozen E equal to value, if not dirty. */ - private volatile E _frozenValue; - - /** True if any mutations have been performed on value. */ - volatile boolean dirty; - - /** The epoch that will follow this one, created on demand. */ - final AtomicReference successorRef = new AtomicReference(null); - - /** A ticket on the successor, released when this epoch is closed. */ - Epoch.Ticket successorTicket; - - /** True if the successor should freeze and clone this epoch's value. */ - boolean freezeRequested; - - private COWEpoch(final boolean mutationAllowed) { - this._activated = new Latch(false); - this.mutationAllowed = mutationAllowed; - } - - public COWEpoch(final E value, final E frozenValue, final int initialSize) { - this._activated = new Latch(true); // pre-triggered - this.mutationAllowed = true; - this.value = value; - this.initialSize = initialSize; - this._frozenValue = frozenValue; - this.dirty = frozenValue == null; - } - - EpochNode attemptInitialArrive() { - return super.attemptArrive(); - } - - @Override - public EpochNode attemptArrive() { - final EpochNode ticket = super.attemptArrive(); - if (ticket != null && !dirty) { - dirty = true; - _frozenValue = null; - } - return ticket; - } - - private void setFrozenValue(final E v) { - if (!dirty) { - _frozenValue = v; - if (dirty) { - _frozenValue = null; - } - } - } - - E getFrozenValue() { - final E v = _frozenValue; - return dirty ? null : v; - } - - protected void onClosed(final int dataSum) { - assert(dataSum == 0 || dirty); - - final COWEpoch succ = successorRef.get(); - if (freezeRequested) { - succ.value = freezeAndClone(value); - succ.setFrozenValue(value); - } - else { - succ.value = value; - if (dirty) { - succ.dirty = true; - } - else { - succ.setFrozenValue(_frozenValue); - } - } - succ.initialSize = initialSize + dataSum; - - _active = succ; - successorTicket.leave(0); - succ._activated.releaseShared(1); - } - - public void awaitActivated() { - _activated.acquireShared(1); - } - - public COWEpoch getOrCreateSuccessor(final boolean preferredMutation) { - final COWEpoch existing = successorRef.get(); - if (existing != null) { - return existing; - } - - final COWEpoch repl = new COWEpoch(preferredMutation); - if (attemptInstallSuccessor(repl)) { - return repl; - } - - return successorRef.get(); - } - - public boolean attemptInstallSuccessor(final COWEpoch succ) { - final Epoch.Ticket t = succ.attemptInitialArrive(); - if (successorRef.compareAndSet(null, succ)) { - successorTicket = t; - beginClose(); - return true; - } - else { - return false; - } - } - } - - private volatile COWEpoch _active; - - /** Creates a new {@link CopyOnWriteManager} holding - * initialValue, with an assumed size of - * initialSize. - */ - public CopyOnWriteManager(final E initialValue, final int initialSize) { - _active = new COWEpoch(initialValue, null, initialSize); - } - - /** The implementing method must mark value as shared, and - * return a new object to use in its place. Hopefully, the majority of - * the work of the clone can be deferred by copy-on-write. - */ - abstract protected E freezeAndClone(final E value); - - /** Returns a clone of a frozen E. */ - abstract protected E cloneFrozen(E frozenValue); - - public CopyOnWriteManager clone() { - final CopyOnWriteManager copy; - try { - copy = (CopyOnWriteManager) super.clone(); - } - catch (final CloneNotSupportedException xx) { - throw new Error("unexpected", xx); - } - - COWEpoch a = _active; - E f = a.getFrozenValue(); - while (f == null) { - a.freezeRequested = true; - final COWEpoch succ = a.getOrCreateSuccessor(a.mutationAllowed); - succ.awaitActivated(); - if (a.value != succ.value) { - f = a.value; - } - a = succ; - } - - copy.createNewEpoch(f, a); - return copy; - } - - private void createNewEpoch(E f, COWEpoch a) - { - _active = new COWEpoch(cloneFrozen(f), f, a.initialSize); - } - - /** Returns a reference to the tree structure suitable for a read - * operation. The returned structure may be mutated by operations that - * have the permission of this {@link CopyOnWriteManager}, but they will - * not observe changes managed by other instances. - */ - public E read() { - return _active.value; - } - - /** Obtains permission to mutate the copy-on-write value held by this - * instance, perhaps blocking while a concurrent snapshot is being - * performed. {@link Epoch.Ticket#leave} must be called exactly once on - * the object returned from this method, after the mutation has been - * completed. The change in size reflected by the mutation should be - * passed as the parameter to leave. - */ - public Epoch.Ticket beginMutation() { - return begin(true); - } - - public Epoch.Ticket beginQuiescent() { - return begin(false); - } - - private Epoch.Ticket begin(final boolean mutation) { - final COWEpoch active = _active; - if (active.mutationAllowed == mutation) { - final Epoch.Ticket ticket = active.attemptArrive(); - if (ticket != null) { - return ticket; - } - } - return begin(mutation, active); - } - - private Epoch.Ticket begin(final boolean mutation, COWEpoch epoch) { - while (true) { - COWEpoch succ = epoch.successorRef.get(); - if (succ == null) { - final COWEpoch newEpoch = new COWEpoch(mutation); - final Epoch.Ticket newTicket = newEpoch.attemptArrive(); - if (epoch.attemptInstallSuccessor(newEpoch)) { - // can't use the ticket until the new epoch is activated - newEpoch.awaitActivated(); - return newTicket; - } - - // if our CAS failed, somebody else succeeded - succ = epoch.successorRef.get(); - } - - // is the successor created by someone else suitable? - if (succ.mutationAllowed == mutation) { - final Epoch.Ticket ticket = succ.attemptArrive(); - if (ticket != null) { - succ.awaitActivated(); - return ticket; - } - } - - epoch = succ; - } - } - - /** Returns a reference to the tree structure suitable for a mutating - * operation. This method may only be called under the protection of a - * ticket returned from {@link #beginMutation}. - */ - public E mutable() { - return _active.value; - } - - /** Returns a reference to a snapshot of this instance's tree structure - * that may be read, but not written. This is accomplished by suspending - * mutation, replacing the mutable root of this manager with the result of - * freezeAndClone(root, false), and then returning a - * reference to the old root. Successive calls to this method may return - * the same instance. - */ - public E frozen() { - COWEpoch a = _active; - E f = a.getFrozenValue(); - while (f == null) { - a.freezeRequested = true; - final COWEpoch succ = a.getOrCreateSuccessor(a.mutationAllowed); - succ.awaitActivated(); - if (a.value != succ.value) { - f = a.value; - } - a = succ; - } - return f; - } - - /** Returns a reference to a snapshot of this instance's tree structure, - * if one is available without requiring any additional copying, otherwise - * returns null. May be used in combination with {@link #beginQuiescent} - * to perform quiescent reads with minimal cost. - */ - public E availableFrozen() { - return _active.getFrozenValue(); - } - - /** Returns true if the computed {@link #size} is zero. */ - public boolean isEmpty() { - // for a different internal implementation (such as a C-SNZI) we might - // be able to do better than this - return size() == 0; - } - - /** Returns the sum of the initialSize parameter passed to the - * constructor, and the size deltas passed to {@link Epoch.Ticket#leave} - * for all of the mutation tickets. The result returned is linearizable - * with mutations, which requires mutation to be quiesced. No tree freeze - * is required, however. - */ - public int size() { - final COWEpoch a = _active; - final Integer delta = a.attemptDataSum(); - if (delta != null) { - return a.initialSize + delta; - } - - // wait for an existing successor, or force one if not already in progress - final COWEpoch succ = a.getOrCreateSuccessor(a.mutationAllowed); - succ.awaitActivated(); - return succ.initialSize; - } -} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/Epoch.java b/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/Epoch.java deleted file mode 100644 index c85320d1f6d20..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/Epoch.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Copyright (c) 2009 Stanford University, unless otherwise specified. - * All rights reserved. - * - * This software was developed by the Pervasive Parallelism Laboratory of - * Stanford University, California, USA. - * - * Permission to use, copy, modify, and distribute this software in source - * or binary form for any purpose with or without fee is hereby granted, - * provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * - * 3. Neither the name of Stanford University nor the names of its - * contributors may be used to endorse or promote products derived - * from this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND - * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY - * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF - * SUCH DAMAGE. - */ - -package org.apache.ignite.internal.util.snaptree; - -/** A Epoch has a lifecycle consisting of three phases: active, - * closing, and closed. During the active phase partipants may arrive and - * leave the epoch. Once a close has been requested, new participants are not - * allowed, only leaving is possible. Once close has been requested and all - * participants have left, the epoch is transitioned to the closed state. - *

      - * Entry is performed with {@link #attemptArrive}, which returns a non-null - * ticket on success or null if {@link #beginClose} has already been called. - * Each successful call to attemptArrive must be paired by a call - * to {@link Ticket#leave} on the returned ticket. - *

      - * The abstract method {@link #onClosed} will be invoked exactly once after - * the epoch becomes closed. It will be passed the sum of the values passed - * to {@link Ticket#leave}. There is no way to query the current participant - * count or state of the epoch without changing it. - *

      - * Internally the epoch responds to contention by increasing its size, - * striping the participant count across multiple objects (and hopefully - * multiple cache lines). Once close has begun, the epoch converts itself to - * a single-shot hierarchical barrier, that also performs a hierarchical - * reduction of the leave parameters. - */ -@SuppressWarnings("ALL") -abstract public class Epoch { - - /** Represents a single successful arrival to an {@link Epoch}. */ - public interface Ticket { - /** Informs the epoch that returned this ticket that the participant - * has left. This method should be called exactly once per ticket. - * The sum of the data values for all tickets will be - * computed and passed to {@link Epoch#onClosed}. - */ - void leave(int data); - } - - private final Root _root = new Root(); - - /** Returns a {@link Ticket} indicating a successful arrival, if no call to - * {@link #beginClose} has been made for this epoch, or returns null if - * close has already begun. {@link Ticket#leave} must be called exactly - * once on any returned ticket. - */ - public Ticket attemptArrive() { - return _root.attemptArrive(); - } - - /** Prevents new arrivals from succeeding, then returns immediately. - * {@link #onClosed} will be called after all outstanding tickets have - * been returned. To block until close is complete, add some sort of - * synchronization logic to the user-defined implementation of {@link - * #onClosed}. - */ - public void beginClose() { - _root.beginClose(); - } - - /** Override this method to provide user-defined behavior. - * dataSum will be the sum of the data values - * passed to {@link Ticket#leave} for all tickets in this epoch. - *

      - * As a simple example, a blocking close operation may be defined by:

      -     *    class BlockingEpoch extends Epoch {
      -     *        private final CountDownLatch _closed = new CountDownLatch(1);
      -     *
      -     *        public void blockingClose() throws InterruptedException {
      -     *            beginClose();
      -     *            _closed.await();
      -     *        }
      -     *
      -     *        protected void onClosed(int dataSum) {
      -     *            _closed.countDown(1);
      -     *        }
      -     *    }
      -     *  
      - */ - abstract protected void onClosed(int dataSum); - - //////////////// debugging stuff - - int computeSpread() { - return _root.computeSpread(); - } - - //////////////// internal implementation - - private class Root extends EpochNode { - /** */ - private static final long serialVersionUID = 0L; - - protected void onClosed(final int dataSum) { - Epoch.this.onClosed(dataSum); - } - } -} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/EpochNode.java b/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/EpochNode.java deleted file mode 100644 index 7ceec453d5af0..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/EpochNode.java +++ /dev/null @@ -1,433 +0,0 @@ -/* - * Copyright (c) 2009 Stanford University, unless otherwise specified. - * All rights reserved. - * - * This software was developed by the Pervasive Parallelism Laboratory of - * Stanford University, California, USA. - * - * Permission to use, copy, modify, and distribute this software in source - * or binary form for any purpose with or without fee is hereby granted, - * provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * - * 3. Neither the name of Stanford University nor the names of its - * contributors may be used to endorse or promote products derived - * from this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND - * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY - * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF - * SUCH DAMAGE. - */ - -package org.apache.ignite.internal.util.snaptree; - -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; - -/** Provides an implementation of the behavior of an {@link Epoch}. */ -@SuppressWarnings("ALL") -abstract class EpochNode extends AtomicLong implements Epoch.Ticket { - - private static final int TRIES_BEFORE_SUBTREE = 2; - private static final int CLOSER_HEAD_START = 1000; - - /** This includes the root. 7 or fewer procs gets 2, 63 or fewer gets - * 3, 511 or fewer 4. We observe that the node count reported by {@link - * #computeSpread} is roughly twice the number of hardware contexts in - * use. - */ - private static final int MAX_LEVELS = 2 + log8(Runtime.getRuntime().availableProcessors()); - - /** Returns floor(log_base_8(value)). */ - private static int log8(final int value) { - return (31 - Integer.numberOfLeadingZeros(value)) / 3; - } - - //////////////// branching factor - - private static final int LOG_BF = 3; - private static final int BF = 1 << LOG_BF; - private static final int BF_MASK = BF - 1; - - //////////////// bit packing - - private static final int DATA_SUM_SHIFT = 32; - private static int dataSum(long state) { return (int)(state >> DATA_SUM_SHIFT); } - private static long withDataDelta(long state, int delta) { return state + (((long) delta) << DATA_SUM_SHIFT); } - - private static final int CHILD_CLOSED_SHIFT = 32 - BF; - private static long ALL_CHILDREN_CLOSED = ((1L << BF) - 1L) << CHILD_CLOSED_SHIFT; - private static long childClosedBit(int which) { return 1L << (CHILD_CLOSED_SHIFT + which); } - private static boolean isChildClosed(long state, int which) { return (state & childClosedBit(which)) != 0; } - private static long withChildClosed(long state, int which, long childState) { - assert(!isChildClosed(state, which)); - return withDataDelta(state | childClosedBit(which), dataSum(childState)); - } - private static boolean isAllChildrenClosed(long state) { return (state & ALL_CHILDREN_CLOSED) == ALL_CHILDREN_CLOSED; } - - private static final int CHILD_PRESENT_SHIFT = CHILD_CLOSED_SHIFT - BF; - private static final long ANY_CHILD_PRESENT = ((1L << BF) - 1L) << CHILD_PRESENT_SHIFT; - private static long childPresentBit(int which) { return 1L << (CHILD_PRESENT_SHIFT + which); } - private static boolean isChildPresent(long state, int which) { return (state & childPresentBit(which)) != 0; } - private static long withChildPresent(long state, int which) { return state | childPresentBit(which); } - private static boolean isAnyChildPresent(long state) { return (state & ANY_CHILD_PRESENT) != 0; } - - private static final long MARK = (1L << (CHILD_PRESENT_SHIFT - 1)); - private static boolean isMarked(long state) { return (state & MARK) != 0L; } - /** Records all non-present children as closed. */ - private static long withMarked(long state) { - final int missingChildren = (~((int) state) >> CHILD_PRESENT_SHIFT) & ((1 << BF) - 1); - return state | MARK | (((long) missingChildren) << CHILD_CLOSED_SHIFT); - } - - private static final long ENTRY_COUNT_MASK = MARK - 1; - private static int entryCount(long state) { return (int) (state & ENTRY_COUNT_MASK); } - private static long withArrive(long state) { return state + 1; } - private static long withLeave(long state, int dataDelta) { return withDataDelta(state - 1, dataDelta); } - private static boolean mayArrive(long state) { return entryCount(state) != ENTRY_COUNT_MASK; } - private static boolean mayLeave(long state) { return entryCount(state) != 0; } - - private static final long CLOSED_MASK = MARK | ALL_CHILDREN_CLOSED | ENTRY_COUNT_MASK; - private static final long CLOSED_VALUE = MARK | ALL_CHILDREN_CLOSED; - private static boolean isClosed(long state) { return (state & CLOSED_MASK) == CLOSED_VALUE; } - - private static final long ENTRY_FAST_PATH_MASK = ANY_CHILD_PRESENT | MARK | (ENTRY_COUNT_MASK - (ENTRY_COUNT_MASK >> 1)); - /** Not marked, no children, and no overflow possible. */ - private static boolean isEntryFastPath(long state) { return (state & ENTRY_FAST_PATH_MASK) == 0L; } - - //////////////// subclasses - - private static class Child extends EpochNode { - /** */ - private static final long serialVersionUID = 0L; - - private Child(final EpochNode parent, final int whichInParent) { - super(parent, whichInParent); - } - - protected void onClosed(final int dataSum) { - throw new Error(); - } - } - - //////////////// instance state - - private static final AtomicReferenceFieldUpdater[] childrenUpdaters = { - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child0"), - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child1"), - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child2"), - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child3"), - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child4"), - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child5"), - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child6"), - AtomicReferenceFieldUpdater.newUpdater(EpochNode.class, EpochNode.class, "_child7") - }; - - private final EpochNode _parent; - private final int _whichInParent; - - // It would be cleaner to use an array of children, but we want to force - // all of the bulk into the same object as the AtomicLong.value. - - // To avoid races between creating a child and marking a node as closed, - // we add a bit to the state for each child that records whether it - // *should* exist. If we find that the bit is set but a child is missing, - // we can create it ourself. - - private volatile EpochNode _child0; - private volatile EpochNode _child1; - private volatile EpochNode _child2; - private volatile EpochNode _child3; - private volatile EpochNode _child4; - private volatile EpochNode _child5; - private volatile EpochNode _child6; - private volatile EpochNode _child7; - - EpochNode() { - _parent = null; - _whichInParent = 0; - } - - private EpochNode(final EpochNode parent, final int whichInParent) { - _parent = parent; - _whichInParent = whichInParent; - } - - //////////////// provided by the caller - - abstract protected void onClosed(int dataSum); - - //////////////// child management - - private EpochNode getChildFromField(final int which) { - switch (which) { - case 0: return _child0; - case 1: return _child1; - case 2: return _child2; - case 3: return _child3; - case 4: return _child4; - case 5: return _child5; - case 6: return _child6; - case 7: return _child7; - default: return null; - } - } - - private EpochNode getChild(final long state, final int which) { - if (!isChildPresent(state, which)) { - return null; - } - final EpochNode existing = getChildFromField(which); - return (existing != null) ? existing : constructPresentChild(which); - } - - @SuppressWarnings("unchecked") - private EpochNode constructPresentChild(final int which) { - final EpochNode n = new Child(this, which); - return childrenUpdaters[which].compareAndSet(this, null, n) ? n : getChildFromField(which); - } - - private EpochNode getOrCreateChild(final int which) { - final EpochNode existing = getChildFromField(which); - return (existing != null) ? existing : createChild(which); - } - - private EpochNode createChild(final int which) { - while (true) { - final long state = get(); - if (isMarked(state)) { - // whatever we've got is what we've got - return getChild(state, which); - } - if (compareAndSet(state, withChildPresent(state, which))) { - // the child now should exist, but we must still actually - // construct and link in the instance - return constructPresentChild(which); - } - } - } - - /** Returns the Node to decr on success, null if - * {@link #beginClose} has already been called on this instance. - */ - public EpochNode attemptArrive() { - final long state = get(); - if (isEntryFastPath(state) && compareAndSet(state, withArrive(state))) { - return this; - } - else { - return attemptArrive(0, 1); - } - } - - private int getIdentity() { - final int h = System.identityHashCode(Thread.currentThread()); - - // Multiply by -127, as suggested by java.util.IdentityHashMap. - // We also set an bit we don't use, to make sure it is never zero. - return (h - (h << 7)) | (1 << 31); - } - - /** level 1 is the root. */ - private EpochNode attemptArrive(int id, final int level) { - int tries = 0; - while (true) { - final long state = get(); - if (isMarked(state)) { - return null; - } - if (isAnyChildPresent(state) || - (tries >= TRIES_BEFORE_SUBTREE && level < MAX_LEVELS)) { - // Go deeper if we have previously detected contention, or if - // we are currently detecting it. Lazy computation of our - // current identity. - if (id == 0) { - id = getIdentity(); - } - final EpochNode child = getOrCreateChild(id & BF_MASK); - if (child == null) { - return null; - } - return child.attemptArrive(id >> LOG_BF, level + 1); - } - if (!mayArrive(state)) { - throw new IllegalStateException("maximum arrival count of " + ENTRY_COUNT_MASK + " exceeded"); - } - if (compareAndSet(state, withArrive(state))) { - // success - return this; - } - - ++tries; - } - } - - /** Should be called on every non-null return value from attemptArrive. */ - public void leave(final int dataDelta) { - while (true) { - final long state = get(); - if (!mayLeave(state)) { - throw new IllegalStateException("incorrect call to Epoch.leave"); - } - final long after = withLeave(state, dataDelta); - if (compareAndSet(state, after)) { - if (isClosed(after)) { - newlyClosed(after); - } - return; - } - } - } - - private void newlyClosed(final long state) { - if (_parent != null) { - // propogate - _parent.childIsNowClosed(_whichInParent, state); - } - else { - // report - onClosed(dataSum(state)); - } - } - - private void childIsNowClosed(final int which, final long childState) { - while (true) { - final long state = get(); - if (isChildClosed(state, which)) { - // not our problem - return; - } - final long after = withChildClosed(state, which, childState); - if (compareAndSet(state, after)) { - if (isClosed(after)) { - newlyClosed(after); - } - return; - } - } - } - - /** Prevents subsequent calls to {@link #attemptArrive} from succeeding. */ - public void beginClose() { - int attempts = 0; - long state; - while (true) { - ++attempts; - - state = get(); - if (isClosed(state)) { - return; - } - - if (isMarked(state)) { - // give the thread that actually performed this transition a - // bit of a head start - if (attempts < CLOSER_HEAD_START) { - continue; - } - break; - } - - // every child that is not present will be recorded as closed by withMarked - final long after = withMarked(state); - if (compareAndSet(state, after)) { - if (isAllChildrenClosed(after)) { - if (isClosed(after) && _parent == null) { - // finished in one CAS, yeah! - onClosed(dataSum(after)); - } - // no second stage necessary - return; - } - // CAS successful, so now we need to beginClose() the children - break; - } - } - - // no new child bits can be set after marking, so this gets everyone - for (int which = 0; which < BF; ++which) { - final EpochNode child = getChild(state, which); - if (child != null) { - child.beginClose(); - } - } - - // Rather than have each child bubble up its closure, we gather it - // here to reduce the number of CASs required. - while (true) { - final long before = get(); - long after = before; - for (int which = 0; which < BF; ++which) { - if (!isChildClosed(before, which)) { - final long childState = getChildFromField(which).get(); - if (isClosed(childState)) { - after = withChildClosed(after, which, childState); - } - } - } - if (before == after) { - return; - } - if (compareAndSet(before, after)) { - if (isClosed(after) && _parent == null) { - onClosed(dataSum(after)); - } - return; - } - } - } - - /** If possible returns the dataSum that would be delivered - * to {@link #onClosed(int)} if this epoch were closed at this moment, - * otherwise returns null. This will succeed if and only if the tree - * consists only of a single node. - */ - public Integer attemptDataSum() { - final long state = get(); - if (!isAnyChildPresent(state) && entryCount(state) == 0) { - // this is better than Integer.valueOf for dynamic escape analysis - //return new Integer(dataSum(state)); - // this is better than new Integer() for object creation - return Integer.valueOf(dataSum(state)); - } - else { - return null; - } - } - - /** For debugging purposes. */ - int computeSpread() { - final long state = get(); - if (isAnyChildPresent(state)) { - int sum = 0; - for (int which = 0; which < BF; ++which) { - final EpochNode child = getChild(state, which); - if (child != null) { - sum += child.computeSpread(); - } - else { - // child would be created for arrive, so count it - sum += 1; - } - } - return sum; - } - else { - return 1; - } - } -} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/SnapTreeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/SnapTreeMap.java deleted file mode 100644 index dce2fb8f24bb3..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/SnapTreeMap.java +++ /dev/null @@ -1,2917 +0,0 @@ -/* - * Copyright (c) 2009 Stanford University, unless otherwise specified. - * All rights reserved. - * - * This software was developed by the Pervasive Parallelism Laboratory of - * Stanford University, California, USA. - * - * Permission to use, copy, modify, and distribute this software in source - * or binary form for any purpose with or without fee is hereby granted, - * provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - * 2. Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * - * 3. Neither the name of Stanford University nor the names of its - * contributors may be used to endorse or promote products derived - * from this software without specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND - * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE - * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL - * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR - * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY - * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF - * SUCH DAMAGE. - */ - -package org.apache.ignite.internal.util.snaptree; - -import org.apache.ignite.*; -import org.apache.ignite.internal.util.*; -import org.apache.ignite.internal.util.lang.*; - -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; -import java.util.AbstractMap; -import java.util.AbstractSet; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.Map; -import java.util.NavigableSet; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.SortedMap; -import java.util.SortedSet; -import java.util.concurrent.ConcurrentNavigableMap; - -// TODO: optimized buildFromSorted -// TODO: submap.clone() - -/** A concurrent AVL tree with fast cloning, based on the algorithm of Bronson, - * Casper, Chafi, and Olukotun, "A Practical Concurrent Binary Search Tree" - * published in PPoPP'10. To simplify the locking protocols rebalancing work - * is performed in pieces, and some removed keys are be retained as routing - * nodes in the tree. - * - *

      This data structure honors all of the contracts of {@link - * java.util.concurrent.ConcurrentSkipListMap}, with the additional contract - * that clone, size, toArray, and iteration are linearizable (atomic). - * - *

      The tree uses optimistic concurrency control. No locks are usually - * required for get, containsKey, firstKey, firstEntry, lastKey, or lastEntry. - * Reads are not lock free (or even obstruction free), but obstructing threads - * perform no memory allocation, system calls, or loops, which seems to work - * okay in practice. All of the updates to the tree are performed in fixed- - * size blocks, so restoration of the AVL balance criteria may occur after a - * change to the tree has linearized (but before the mutating operation has - * returned). The tree is always properly balanced when quiescent. - * - *

      To clone the tree (or produce a snapshot for consistent iteration) the - * root node is marked as shared, which must be (*) done while there are no - * pending mutations. New mutating operations are blocked if a mark is - * pending, and when existing mutating operations are completed the mark is - * made. - * * - It would be less disruptive if we immediately marked the root as - * shared, and then waited for pending operations that might not have seen the - * mark without blocking new mutations. This could result in imbalance being - * frozen into the shared portion of the tree, though. To minimize the - * problem we perform the mark and reenable mutation on whichever thread - * notices that the entry count has become zero, to reduce context switches on - * the critical path. - * - *

      The same multi-cache line data structure required for efficiently - * tracking the entry and exit for mutating operations is used to maintain the - * current size of the tree. This means that the size can be computed by - * quiescing as for a clone, but without doing any marking. - * - *

      Range queries such as higherKey are not amenable to the optimistic - * hand-over-hand locking scheme used for exact searches, so they are - * implemented with pessimistic concurrency control. Mutation can be - * considered to acquire a lock on the map in Intention-eXclusive mode, range - * queries, size(), and root marking acquire the lock in Shared mode. - * - * @author Nathan Bronson - */ -@SuppressWarnings("ALL") -public class SnapTreeMap extends AbstractMap implements ConcurrentNavigableMap, Cloneable, - Serializable { - - /** */ - private static final long serialVersionUID = 0L; - - /** If false, null values will trigger a NullPointerException. When false, - * this map acts exactly like a ConcurrentSkipListMap, except for the - * running time of the methods. The ability to get a snapshot reduces the - * potential ambiguity between null values and absent entries, so I'm not - * sure what the default should be. - */ - static final boolean AllowNullValues = false; - - /** This is a special value that indicates the presence of a null value, - * to differentiate from the absence of a value. Only used when - * {@link #AllowNullValues} is true. - */ - static final Object SpecialNull = new Object(); - - /** This is a special value that indicates that an optimistic read - * failed. - */ - static final Object SpecialRetry = new Object(); - - - /** The number of spins before yielding. */ - static final int SpinCount = Integer.parseInt(System.getProperty("snaptree.spin", "100")); - - /** The number of yields before blocking. */ - static final int YieldCount = Integer.parseInt(System.getProperty("snaptree.yield", "0")); - - - // we encode directions as characters - static final char Left = 'L'; - static final char Right = 'R'; - - - /** An OVL is a version number and lock used for optimistic - * concurrent control of some program invariant. If {@link #isShrinking} - * then the protected invariant is changing. If two reads of an OVL are - * performed that both see the same non-changing value, the reader may - * conclude that no changes to the protected invariant occurred between - * the two reads. The special value UnlinkedOVL is not changing, and is - * guaranteed to not result from a normal sequence of beginChange and - * endChange operations. - *

      - * For convenience endChange(ovl) == endChange(beginChange(ovl)). - */ - static long beginChange(long ovl) { return ovl | 1; } - static long endChange(long ovl) { return (ovl | 3) + 1; } - static final long UnlinkedOVL = 2; - - static boolean isShrinking(long ovl) { return (ovl & 1) != 0; } - static boolean isUnlinked(long ovl) { return (ovl & 2) != 0; } - static boolean isShrinkingOrUnlinked(long ovl) { return (ovl & 3) != 0L; } - - - protected static class Node implements Map.Entry { - public K key; - volatile int height; - - /** null means this node is conceptually not present in the map. - * SpecialNull means the value is null. - */ - volatile Object vOpt; - volatile Node parent; - volatile long shrinkOVL; - volatile Node left; - volatile Node right; - - Node(final K key, - final int height, - final Object vOpt, - final Node parent, - final long shrinkOVL, - final Node left, - final Node right) - { - this.key = key; - this.height = height; - this.vOpt = vOpt; - this.parent = parent; - this.shrinkOVL = shrinkOVL; - this.left = left; - this.right = right; - } - - @Override - public K getKey() { return key; } - - @Override - @SuppressWarnings("unchecked") - public V getValue() { - final Object tmp = vOpt; - if (AllowNullValues) { - return tmp == SpecialNull ? null : (V)tmp; - } else { - return (V)tmp; - } - } - - @Override - public V setValue(final V v) { - throw new UnsupportedOperationException(); - } - - Node child(char dir) { return dir == Left ? left : right; } - - void setChild(char dir, Node node) { - if (dir == Left) { - left = node; - } else { - right = node; - } - } - - //////// copy-on-write stuff - - private static boolean isShared(final Node node) { - return node != null && node.parent == null; - } - - static Node markShared(final Node node) { - if (node != null) { - node.parent = null; - } - return node; - } - - private Node lazyCopy(Node newParent) { - assert (isShared(this)); - assert (!isShrinkingOrUnlinked(shrinkOVL)); - - return new Node(key, height, vOpt, newParent, 0L, markShared(left), markShared(right)); - } - - Node unsharedLeft() { - final Node cl = left; - if (!isShared(cl)) { - return cl; - } else { - lazyCopyChildren(); - return left; - } - } - - Node unsharedRight() { - final Node cr = right; - if (!isShared(cr)) { - return cr; - } else { - lazyCopyChildren(); - return right; - } - } - - Node unsharedChild(final char dir) { - return dir == Left ? unsharedLeft() : unsharedRight(); - } - - private synchronized void lazyCopyChildren() { - final Node cl = left; - if (isShared(cl)) { - left = cl.lazyCopy(this); - } - final Node cr = right; - if (isShared(cr)) { - right = cr.lazyCopy(this); - } - } - - //////// per-node blocking - - private void waitUntilShrinkCompleted(final long ovl) { - if (!isShrinking(ovl)) { - return; - } - - for (int tries = 0; tries < SpinCount; ++tries) { - if (shrinkOVL != ovl) { - return; - } - } - - for (int tries = 0; tries < YieldCount; ++tries) { - Thread.yield(); - if (shrinkOVL != ovl) { - return; - } - } - - // spin and yield failed, use the nuclear option - synchronized (this) { - // we can't have gotten the lock unless the shrink was over - } - assert(shrinkOVL != ovl); - } - - int validatedHeight() { - final int hL = left == null ? 0 : left.validatedHeight(); - final int hR = right == null ? 0 : right.validatedHeight(); - assert(Math.abs(hL - hR) <= 1); - final int h = 1 + Math.max(hL, hR); - assert(h == height); - return height; - } - - //////// SubMap.size() helper - - static int computeFrozenSize(Node root, - Comparable fromCmp, - boolean fromIncl, - final Comparable toCmp, - final boolean toIncl) { - int result = 0; - while (true) { - if (root == null) { - return result; - } - if (fromCmp != null) { - final int c = fromCmp.compareTo(root.key); - if (c > 0 || (c == 0 && !fromIncl)) { - // all matching nodes are on the right side - root = root.right; - continue; - } - } - if (toCmp != null) { - final int c = toCmp.compareTo(root.key); - if (c < 0 || (c == 0 && !toIncl)) { - // all matching nodes are on the left side - root = root.left; - continue; - } - } - - // Current node matches. Nodes on left no longer need toCmp, nodes - // on right no longer need fromCmp. - if (root.vOpt != null) { - ++result; - } - result += computeFrozenSize(root.left, fromCmp, fromIncl, null, false); - fromCmp = null; - root = root.right; - } - } - - //////// Map.Entry stuff - - @Override - public boolean equals(final Object o) { - if (!(o instanceof Map.Entry)) { - return false; - } - final Map.Entry rhs = (Map.Entry)o; - return eq(key, rhs.getKey()) && eq(getValue(), rhs.getValue()); - } - - private static boolean eq(final Object o1, final Object o2) { - return o1 == null ? o2 == null : o1.equals(o2); - } - - @Override - public int hashCode() { - return (key == null ? 0 : key.hashCode()) ^ - (getValue() == null ? 0 : getValue().hashCode()); - } - - @Override - public String toString() { - return key + "=" + getValue(); - } - } - - private static class RootHolder extends Node { - RootHolder() { - super(null, 1, null, null, 0L, null, null); - } - - RootHolder(final RootHolder snapshot) { - super(null, 1 + snapshot.height, null, null, 0L, null, snapshot.right); - } - } - - private static class COWMgr extends CopyOnWriteManager> { - COWMgr() { - super(new RootHolder(), 0); - } - - COWMgr(final RootHolder initialValue, final int initialSize) { - super(initialValue, initialSize); - } - - protected RootHolder freezeAndClone(final RootHolder value) { - Node.markShared(value.right); - return new RootHolder(value); - } - - protected RootHolder cloneFrozen(final RootHolder frozenValue) { - return new RootHolder(frozenValue); - } - } - - //////// node access functions - - private static int height(final Node node) { - return node == null ? 0 : node.height; - } - - @SuppressWarnings("unchecked") - private V decodeNull(final Object vOpt) { - assert (vOpt != SpecialRetry); - if (AllowNullValues) { - return vOpt == SpecialNull ? null : (V)vOpt; - } else { - return (V)vOpt; - } - } - - private static Object encodeNull(final Object v) { - if (AllowNullValues) { - return v == null ? SpecialNull : v; - } else { - if (v == null) { - throw new NullPointerException(); - } - return v; - } - } - - //////////////// state - - private final Comparator comparator; - private transient volatile COWMgr holderRef; - - //////////////// public interface - - public SnapTreeMap() { - this.comparator = null; - this.holderRef = new COWMgr(); - } - - public SnapTreeMap(final Comparator comparator) { - this.comparator = comparator; - this.holderRef = new COWMgr(); - } - - public SnapTreeMap(final Map source) { - this.comparator = null; - this.holderRef = new COWMgr(); - putAll(source); - } - - public SnapTreeMap(final SortedMap source) { - this.comparator = source.comparator(); - if (source instanceof SnapTreeMap) { - final SnapTreeMap s = (SnapTreeMap) source; - this.holderRef = (COWMgr) s.holderRef.clone(); - } - else { - // TODO: take advantage of the sort order - // for now we optimize only by bypassing the COWMgr - int size = 0; - final RootHolder holder = new RootHolder(); - for (Map.Entry e : source.entrySet()) { - final K k = e.getKey(); - final V v = e.getValue(); - if (k == null) { - throw new NullPointerException("source map contained a null key"); - } - if (!AllowNullValues && v == null) { - throw new NullPointerException("source map contained a null value"); - } - updateUnderRoot(k, comparable(k), UpdateAlways, null, encodeNull(v), holder); - ++size; - } - - this.holderRef = new COWMgr(holder, size); - } - } - - @SuppressWarnings("unchecked") - @Override - public SnapTreeMap clone() { - final SnapTreeMap copy; - try { - copy = (SnapTreeMap) super.clone(); - } catch (final CloneNotSupportedException xx) { - throw new InternalError(); - } - assert(copy.comparator == comparator); - copy.holderRef = (COWMgr) holderRef.clone(); - return copy; - } - - @Override - public int size() { - return holderRef.size(); - } - - @Override - public boolean isEmpty() { - // removed-but-not-unlinked nodes cannot be leaves, so if the tree is - // truly empty then the root holder has no right child - return holderRef.read().right == null; - } - - @Override - public void clear() { - holderRef = new COWMgr(); - } - - @Override - public Comparator comparator() { - return comparator; - } - - @Override - public boolean containsValue(final Object value) { - // apply the same null policy as the rest of the code, but fall - // back to the default implementation - encodeNull(value); - return super.containsValue(value); - } - - //////// concurrent search - - @Override - public boolean containsKey(final Object key) { - return getImpl(key) != null; - } - - @Override - public V get(final Object key) { - return decodeNull(getImpl(key)); - } - - @SuppressWarnings("unchecked") - protected Comparable comparable(final Object key) { - if (key == null) { - throw new NullPointerException(); - } - if (comparator == null) { - return (Comparable)key; - } - return new Comparable() { - final Comparator _cmp = comparator; - - @SuppressWarnings("unchecked") - public int compareTo(final K rhs) { return _cmp.compare((K)key, rhs); } - }; - } - - /** Returns either a value or SpecialNull, if present, or null, if absent. */ - private Object getImpl(final Object key) { - final Comparable k = comparable(key); - - while (true) { - final Node right = holderRef.read().right; - if (right == null) { - return null; - } else { - final int rightCmp = k.compareTo(right.key); - if (rightCmp == 0) { - // who cares how we got here - return right.vOpt; - } - - final long ovl = right.shrinkOVL; - if (isShrinkingOrUnlinked(ovl)) { - right.waitUntilShrinkCompleted(ovl); - // RETRY - } else if (right == holderRef.read().right) { - // the reread of .right is the one protected by our read of ovl - final Object vo = attemptGet(k, right, (rightCmp < 0 ? Left : Right), ovl); - if (vo != SpecialRetry) { - return vo; - } - // else RETRY - } - } - } - } - - private Object attemptGet(final Comparable k, - final Node node, - final char dirToC, - final long nodeOVL) { - while (true) { - final Node child = node.child(dirToC); - - if (child == null) { - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - - // Note is not present. Read of node.child occurred while - // parent.child was valid, so we were not affected by any - // shrinks. - return null; - } else { - final int childCmp = k.compareTo(child.key); - if (childCmp == 0) { - // how we got here is irrelevant - return child.vOpt; - } - - // child is non-null - final long childOVL = child.shrinkOVL; - if (isShrinkingOrUnlinked(childOVL)) { - child.waitUntilShrinkCompleted(childOVL); - - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - // else RETRY - } else if (child != node.child(dirToC)) { - // this .child is the one that is protected by childOVL - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - // else RETRY - } else { - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - - // At this point we know that the traversal our parent took - // to get to node is still valid. The recursive - // implementation will validate the traversal from node to - // child, so just prior to the nodeOVL validation both - // traversals were definitely okay. This means that we are - // no longer vulnerable to node shrinks, and we don't need - // to validate nodeOVL any more. - final Object vo = attemptGet(k, child, (childCmp < 0 ? Left : Right), childOVL); - if (vo != SpecialRetry) { - return vo; - } - // else RETRY - } - } - } - } - - @Override - public K firstKey() { - return extremeKeyOrThrow(Left); - } - - @Override - @SuppressWarnings("unchecked") - public Map.Entry firstEntry() { - return (SimpleImmutableEntry) extreme(false, Left); - } - - @Override - public K lastKey() { - return extremeKeyOrThrow(Right); - } - - @SuppressWarnings("unchecked") - public Map.Entry lastEntry() { - return (SimpleImmutableEntry) extreme(false, Right); - } - - private K extremeKeyOrThrow(final char dir) { - final K k = (K) extreme(true, dir); - if (k == null) { - throw new NoSuchElementException(); - } - return k; - } - - /** Returns a key if returnKey is true, a SimpleImmutableEntry otherwise. - * Returns null if none exists. - */ - private Object extreme(final boolean returnKey, final char dir) { - while (true) { - final Node right = holderRef.read().right; - if (right == null) { - return null; - } else { - final long ovl = right.shrinkOVL; - if (isShrinkingOrUnlinked(ovl)) { - right.waitUntilShrinkCompleted(ovl); - // RETRY - } else if (right == holderRef.read().right) { - // the reread of .right is the one protected by our read of ovl - final Object vo = attemptExtreme(returnKey, dir, right, ovl); - if (vo != SpecialRetry) { - return vo; - } - // else RETRY - } - } - } - } - - private Object attemptExtreme(final boolean returnKey, - final char dir, - final Node node, - final long nodeOVL) { - while (true) { - final Node child = node.child(dir); - - if (child == null) { - // read of the value must be protected by the OVL, because we - // must linearize against another thread that inserts a new min - // key and then changes this key's value - final Object vo = node.vOpt; - - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - - assert(vo != null); - - return returnKey ? node.key : new SimpleImmutableEntry(node.key, decodeNull(vo)); - } else { - // child is non-null - final long childOVL = child.shrinkOVL; - if (isShrinkingOrUnlinked(childOVL)) { - child.waitUntilShrinkCompleted(childOVL); - - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - // else RETRY - } else if (child != node.child(dir)) { - // this .child is the one that is protected by childOVL - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - // else RETRY - } else { - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - - final Object vo = attemptExtreme(returnKey, dir, child, childOVL); - if (vo != SpecialRetry) { - return vo; - } - // else RETRY - } - } - } - } - - //////////////// quiesced search - - @Override - @SuppressWarnings("unchecked") - public K lowerKey(final K key) { - return (K) boundedExtreme(null, false, comparable(key), false, true, Right); - } - - @Override - @SuppressWarnings("unchecked") - public K floorKey(final K key) { - return (K) boundedExtreme(null, false, comparable(key), true, true, Right); - } - - @Override - @SuppressWarnings("unchecked") - public K ceilingKey(final K key) { - return (K) boundedExtreme(comparable(key), true, null, false, true, Left); - } - - @Override - @SuppressWarnings("unchecked") - public K higherKey(final K key) { - return (K) boundedExtreme(comparable(key), false, null, false, true, Left); - } - - - @Override - @SuppressWarnings("unchecked") - public Entry lowerEntry(final K key) { - return (Entry) boundedExtreme(null, false, comparable(key), false, false, Right); - } - - @Override - @SuppressWarnings("unchecked") - public Entry floorEntry(final K key) { - return (Entry) boundedExtreme(null, false, comparable(key), true, false, Right); - } - - @Override - @SuppressWarnings("unchecked") - public Entry ceilingEntry(final K key) { - return (Entry) boundedExtreme(comparable(key), true, null, false, false, Left); - } - - @Override - @SuppressWarnings("unchecked") - public Entry higherEntry(final K key) { - return (Entry) boundedExtreme(comparable(key), false, null, false, false, Left); - } - - /** Returns null if none exists. */ - @SuppressWarnings("unchecked") - private K boundedExtremeKeyOrThrow(final Comparable minCmp, - final boolean minIncl, - final Comparable maxCmp, - final boolean maxIncl, - final char dir) { - final K k = (K) boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, true, dir); - if (k == null) { - throw new NoSuchElementException(); - } - return k; - } - - /** Returns null if none exists. */ - @SuppressWarnings("unchecked") - private Object boundedExtreme(final Comparable minCmp, - final boolean minIncl, - final Comparable maxCmp, - final boolean maxIncl, - final boolean returnKey, - final char dir) { - K resultKey; - Object result; - - if ((dir == Left && minCmp == null) || (dir == Right && maxCmp == null)) { - // no bound in the extreme direction, so use the concurrent search - result = extreme(returnKey, dir); - if (result == null) { - return null; - } - resultKey = returnKey ? (K) result : ((SimpleImmutableEntry) result).getKey(); - } - else { - RootHolder holder = holderRef.availableFrozen(); - final Epoch.Ticket ticket; - if (holder == null) { - ticket = holderRef.beginQuiescent(); - holder = holderRef.read(); - } - else { - ticket = null; - } - try { - final Node node = (dir == Left) - ? boundedMin(holder.right, minCmp, minIncl) - : boundedMax(holder.right, maxCmp, maxIncl); - if (node == null) { - return null; - } - resultKey = node.key; - if (returnKey) { - result = node.key; - } - else if (ticket == null) { - // node of a frozen tree is okay, copy otherwise - result = node; - } - else { - // we must copy the node - result = new SimpleImmutableEntry(node.key, node.getValue()); - } - } - finally { - if (ticket != null) { - ticket.leave(0); - } - } - } - - if (dir == Left && maxCmp != null) { - final int c = maxCmp.compareTo(resultKey); - if (c < 0 || (c == 0 && !maxIncl)) { - return null; - } - } - if (dir == Right && minCmp != null) { - final int c = minCmp.compareTo(resultKey); - if (c > 0 || (c == 0 && !minIncl)) { - return null; - } - } - - return result; - } - - private Node boundedMin(Node node, - final Comparable minCmp, - final boolean minIncl) { - while (node != null) { - final int c = minCmp.compareTo(node.key); - if (c < 0) { - // there may be a matching node on the left branch - final Node z = boundedMin(node.left, minCmp, minIncl); - if (z != null) { - return z; - } - } - - if (c < 0 || (c == 0 && minIncl)) { - // this node is a candidate, is it actually present? - if (node.vOpt != null) { - return node; - } - } - - // the matching node is on the right branch if it is present - node = node.right; - } - return null; - } - - private Node boundedMax(Node node, - final Comparable maxCmp, - final boolean maxIncl) { - while (node != null) { - final int c = maxCmp.compareTo(node.key); - if (c > 0) { - // there may be a matching node on the right branch - final Node z = boundedMax(node.right, maxCmp, maxIncl); - if (z != null) { - return z; - } - } - - if (c > 0 || (c == 0 && maxIncl)) { - // this node is a candidate, is it actually present? - if (node.vOpt != null) { - return node; - } - } - - // the matching node is on the left branch if it is present - node = node.left; - } - return null; - } - - //////////////// update - - private static final int UpdateAlways = 0; - private static final int UpdateIfAbsent = 1; - private static final int UpdateIfPresent = 2; - private static final int UpdateIfEq = 3; - - private static boolean shouldUpdate(final int func, final Object prev, final Object expected) { - switch (func) { - case UpdateAlways: return true; - case UpdateIfAbsent: return prev == null; - case UpdateIfPresent: return prev != null; - default: { // UpdateIfEq - assert(expected != null); - if (prev == null) { - return false; - } - if (AllowNullValues && (prev == SpecialNull || expected == SpecialNull)) { - return prev == SpecialNull && expected == SpecialNull; - } - return prev.equals(expected); - } - } - } - - private static Object noUpdateResult(final int func, final Object prev) { - return func == UpdateIfEq ? Boolean.FALSE : prev; - } - - private static Object updateResult(final int func, final Object prev) { - return func == UpdateIfEq ? Boolean.TRUE : prev; - } - - private static int sizeDelta(final int func, final Object result, final Object newValue) { - switch (func) { - case UpdateAlways: { - return (result != null ? -1 : 0) + (newValue != null ? 1 : 0); - } - case UpdateIfAbsent: { - assert(newValue != null); - return result != null ? 0 : 1; - } - case UpdateIfPresent: { - return result == null ? 0 : (newValue != null ? 0 : -1); - } - default: { // UpdateIfEq - return !((Boolean) result) ? 0 : (newValue != null ? 0 : -1); - } - } - } - - @Override - public V put(final K key, final V value) { - return decodeNull(update(key, UpdateAlways, null, encodeNull(value))); - } - - @Override - public V putIfAbsent(final K key, final V value) { - return decodeNull(update(key, UpdateIfAbsent, null, encodeNull(value))); - } - - @Override - public V replace(final K key, final V value) { - return decodeNull(update(key, UpdateIfPresent, null, encodeNull(value))); - } - - @Override - public boolean replace(final K key, final V oldValue, final V newValue) { - return (Boolean) update(key, UpdateIfEq, encodeNull(oldValue), encodeNull(newValue)); - } - - @Override - public V remove(final Object key) { - return decodeNull(update(key, UpdateAlways, null, null)); - } - - @Override - public boolean remove(final Object key, final Object value) { - if (key == null) { - throw new NullPointerException(); - } - if (!AllowNullValues && value == null) { - return false; - } - return (Boolean) update(key, UpdateIfEq, encodeNull(value), null); - } - - // manages the epoch - private Object update(final Object key, - final int func, - final Object expected, - final Object newValue) { - final Comparable k = comparable(key); - int sd = 0; - final Epoch.Ticket ticket = holderRef.beginMutation(); - try { - final Object result = updateUnderRoot(key, k, func, expected, newValue, holderRef.mutable()); - sd = sizeDelta(func, result, newValue); - return result; - } finally { - ticket.leave(sd); - } - } - - // manages updates to the root holder - @SuppressWarnings("unchecked") - private Object updateUnderRoot(final Object key, - final Comparable k, - final int func, - final Object expected, - final Object newValue, - final RootHolder holder) { - - while (true) { - final Node right = holder.unsharedRight(); - if (right == null) { - // key is not present - if (!shouldUpdate(func, null, expected)) { - return noUpdateResult(func, null); - } - if (newValue == null || attemptInsertIntoEmpty((K)key, newValue, holder)) { - // nothing needs to be done, or we were successful, prev value is Absent - return updateResult(func, null); - } - // else RETRY - } else { - final long ovl = right.shrinkOVL; - if (isShrinkingOrUnlinked(ovl)) { - right.waitUntilShrinkCompleted(ovl); - // RETRY - } else if (right == holder.right) { - // this is the protected .right - final Object vo = attemptUpdate(key, k, func, expected, newValue, holder, right, ovl); - if (vo != SpecialRetry) { - return vo; - } - // else RETRY - } - } - } - } - - private boolean attemptInsertIntoEmpty(final K key, - final Object vOpt, - final RootHolder holder) { - synchronized (holder) { - if (holder.right == null) { - holder.right = new Node(key, 1, vOpt, holder, 0L, null, null); - holder.height = 2; - return true; - } else { - return false; - } - } - } - - /** If successful returns the non-null previous value, SpecialNull for a - * null previous value, or null if not previously in the map. - * The caller should retry if this method returns SpecialRetry. - */ - @SuppressWarnings("unchecked") - private Object attemptUpdate(final Object key, - final Comparable k, - final int func, - final Object expected, - final Object newValue, - final Node parent, - final Node node, - final long nodeOVL) { - // As the search progresses there is an implicit min and max assumed for the - // branch of the tree rooted at node. A left rotation of a node x results in - // the range of keys in the right branch of x being reduced, so if we are at a - // node and we wish to traverse to one of the branches we must make sure that - // the node has not undergone a rotation since arriving from the parent. - // - // A rotation of node can't screw us up once we have traversed to node's - // child, so we don't need to build a huge transaction, just a chain of - // smaller read-only transactions. - - assert (nodeOVL != UnlinkedOVL); - - final int cmp = k.compareTo(node.key); - if (cmp == 0) { - return attemptNodeUpdate(func, expected, newValue, parent, node); - } - - final char dirToC = cmp < 0 ? Left : Right; - - while (true) { - final Node child = node.unsharedChild(dirToC); - - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - - if (child == null) { - // key is not present - if (newValue == null) { - // Removal is requested. Read of node.child occurred - // while parent.child was valid, so we were not affected - // by any shrinks. - return noUpdateResult(func, null); - } else { - // Update will be an insert. - final boolean success; - final Node damaged; - synchronized (node) { - // Validate that we haven't been affected by past - // rotations. We've got the lock on node, so no future - // rotations can mess with us. - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - - if (node.child(dirToC) != null) { - // Lost a race with a concurrent insert. No need - // to back up to the parent, but we must RETRY in - // the outer loop of this method. - success = false; - damaged = null; - } else { - // We're valid. Does the user still want to - // perform the operation? - if (!shouldUpdate(func, null, expected)) { - return noUpdateResult(func, null); - } - - // Create a new leaf - node.setChild(dirToC, new Node((K)key, 1, newValue, node, 0L, null, null)); - success = true; - - // attempt to fix node.height while we've still got - // the lock - damaged = fixHeight_nl(node); - } - } - if (success) { - fixHeightAndRebalance(damaged); - return updateResult(func, null); - } - // else RETRY - } - } else { - // non-null child - final long childOVL = child.shrinkOVL; - if (isShrinkingOrUnlinked(childOVL)) { - child.waitUntilShrinkCompleted(childOVL); - // RETRY - } else if (child != node.child(dirToC)) { - // this second read is important, because it is protected - // by childOVL - // RETRY - } else { - // validate the read that our caller took to get to node - if (node.shrinkOVL != nodeOVL) { - return SpecialRetry; - } - - // At this point we know that the traversal our parent took - // to get to node is still valid. The recursive - // implementation will validate the traversal from node to - // child, so just prior to the nodeOVL validation both - // traversals were definitely okay. This means that we are - // no longer vulnerable to node shrinks, and we don't need - // to validate nodeOVL any more. - final Object vo = attemptUpdate(key, k, func, expected, newValue, node, child, childOVL); - if (vo != SpecialRetry) { - return vo; - } - // else RETRY - } - } - } - } - - /** parent will only be used for unlink, update can proceed even if parent - * is stale. - */ - private Object attemptNodeUpdate(final int func, - final Object expected, - final Object newValue, - final Node parent, - final Node node) { - if (newValue == null) { - // removal - if (node.vOpt == null) { - // This node is already removed, nothing to do. - return noUpdateResult(func, null); - } - } - - if (newValue == null && (node.left == null || node.right == null)) { - // potential unlink, get ready by locking the parent - final Object prev; - final Node damaged; - synchronized (parent) { - if (isUnlinked(parent.shrinkOVL) || node.parent != parent) { - return SpecialRetry; - } - - synchronized (node) { - prev = node.vOpt; - if (!shouldUpdate(func, prev, expected)) { - return noUpdateResult(func, prev); - } - if (prev == null) { - return updateResult(func, prev); - } - if (!attemptUnlink_nl(parent, node)) { - return SpecialRetry; - } - } - // try to fix the parent while we've still got the lock - damaged = fixHeight_nl(parent); - } - fixHeightAndRebalance(damaged); - return updateResult(func, prev); - } else { - // potential update (including remove-without-unlink) - synchronized (node) { - // regular version changes don't bother us - if (isUnlinked(node.shrinkOVL)) { - return SpecialRetry; - } - - final Object prev = node.vOpt; - if (!shouldUpdate(func, prev, expected)) { - return noUpdateResult(func, prev); - } - - // retry if we now detect that unlink is possible - if (newValue == null && (node.left == null || node.right == null)) { - return SpecialRetry; - } - - // update in-place - node.vOpt = newValue; - - afterNodeUpdate_nl(node, newValue); - - return updateResult(func, prev); - } - } - } - - protected void afterNodeUpdate_nl(Node node, Object val) { - } - - /** Does not adjust the size or any heights. */ - private boolean attemptUnlink_nl(final Node parent, final Node node) { - // assert (Thread.holdsLock(parent)); - // assert (Thread.holdsLock(node)); - assert (!isUnlinked(parent.shrinkOVL)); - - final Node parentL = parent.left; - final Node parentR = parent.right; - if (parentL != node && parentR != node) { - // node is no longer a child of parent - return false; - } - - assert (!isUnlinked(node.shrinkOVL)); - assert (parent == node.parent); - - final Node left = node.unsharedLeft(); - final Node right = node.unsharedRight(); - if (left != null && right != null) { - // splicing is no longer possible - return false; - } - final Node splice = left != null ? left : right; - - if (parentL == node) { - parent.left = splice; - } else { - parent.right = splice; - } - if (splice != null) { - splice.parent = parent; - } - - node.shrinkOVL = UnlinkedOVL; - node.vOpt = null; - - return true; - } - - //////////////// NavigableMap stuff - - @Override - public Map.Entry pollFirstEntry() { - return pollExtremeEntry(Left); - } - - @Override - public Map.Entry pollLastEntry() { - return pollExtremeEntry(Right); - } - - private Map.Entry pollExtremeEntry(final char dir) { - final Epoch.Ticket ticket = holderRef.beginMutation(); - int sizeDelta = 0; - try { - final Map.Entry prev = pollExtremeEntryUnderRoot(dir, holderRef.mutable()); - if (prev != null) { - sizeDelta = -1; - } - return prev; - } finally { - ticket.leave(sizeDelta); - } - } - - private Map.Entry pollExtremeEntryUnderRoot(final char dir, final RootHolder holder) { - while (true) { - final Node right = holder.unsharedRight(); - if (right == null) { - // tree is empty, nothing to remove - return null; - } else { - final long ovl = right.shrinkOVL; - if (isShrinkingOrUnlinked(ovl)) { - right.waitUntilShrinkCompleted(ovl); - // RETRY - } else if (right == holder.right) { - // this is the protected .right - final Map.Entry result = attemptRemoveExtreme(dir, holder, right, ovl); - if (result != SpecialRetry) { - return result; - } - // else RETRY - } - } - } - } - - private Map.Entry attemptRemoveExtreme(final char dir, - final Node parent, - final Node node, - final long nodeOVL) { - assert (nodeOVL != UnlinkedOVL); - - while (true) { - final Node child = node.unsharedChild(dir); - - if (nodeOVL != node.shrinkOVL) { - return null; - } - - if (child == null) { - // potential unlink, get ready by locking the parent - final Object vo; - final Node damaged; - synchronized (parent) { - if (isUnlinked(parent.shrinkOVL) || node.parent != parent) { - return null; - } - - synchronized (node) { - vo = node.vOpt; - if (node.child(dir) != null || !attemptUnlink_nl(parent, node)) { - return null; - } - // success! - } - // try to fix parent.height while we've still got the lock - damaged = fixHeight_nl(parent); - } - fixHeightAndRebalance(damaged); - return new SimpleImmutableEntry(node.key, decodeNull(vo)); - } else { - // keep going down - final long childOVL = child.shrinkOVL; - if (isShrinkingOrUnlinked(childOVL)) { - child.waitUntilShrinkCompleted(childOVL); - // RETRY - } else if (child != node.child(dir)) { - // this second read is important, because it is protected - // by childOVL - // RETRY - } else { - // validate the read that our caller took to get to node - if (node.shrinkOVL != nodeOVL) { - return null; - } - - final Map.Entry result = attemptRemoveExtreme(dir, node, child, childOVL); - if (result != null) { - return result; - } - // else RETRY - } - } - } - } - - - - //////////////// tree balance and height info repair - - private static final int UnlinkRequired = -1; - private static final int RebalanceRequired = -2; - private static final int NothingRequired = -3; - - private int nodeCondition(final Node node) { - // Begin atomic. - - final Node nL = node.left; - final Node nR = node.right; - - if ((nL == null || nR == null) && node.vOpt == null) { - return UnlinkRequired; - } - - final int hN = node.height; - final int hL0 = height(nL); - final int hR0 = height(nR); - - // End atomic. Since any thread that changes a node promises to fix - // it, either our read was consistent (and a NothingRequired conclusion - // is correct) or someone else has taken responsibility for either node - // or one of its children. - - final int hNRepl = 1 + Math.max(hL0, hR0); - final int bal = hL0 - hR0; - - if (bal < -1 || bal > 1) { - return RebalanceRequired; - } - - return hN != hNRepl ? hNRepl : NothingRequired; - } - - private void fixHeightAndRebalance(Node node) { - while (node != null && node.parent != null) { - final int condition = nodeCondition(node); - if (condition == NothingRequired || isUnlinked(node.shrinkOVL)) { - // nothing to do, or no point in fixing this node - return; - } - - if (condition != UnlinkRequired && condition != RebalanceRequired) { - synchronized (node) { - node = fixHeight_nl(node); - } - } else { - final Node nParent = node.parent; - synchronized (nParent) { - if (!isUnlinked(nParent.shrinkOVL) && node.parent == nParent) { - synchronized (node) { - node = rebalance_nl(nParent, node); - } - } - // else RETRY - } - } - } - } - - /** Attempts to fix the height of a (locked) damaged node, returning the - * lowest damaged node for which this thread is responsible. Returns null - * if no more repairs are needed. - */ - private Node fixHeight_nl(final Node node) { - final int c = nodeCondition(node); - switch (c) { - case RebalanceRequired: - case UnlinkRequired: - // can't repair - return node; - case NothingRequired: - // Any future damage to this node is not our responsibility. - return null; - default: - node.height = c; - // we've damaged our parent, but we can't fix it now - return node.parent; - } - } - - /** nParent and n must be locked on entry. Returns a damaged node, or null - * if no more rebalancing is necessary. - */ - private Node rebalance_nl(final Node nParent, final Node n) { - - final Node nL = n.unsharedLeft(); - final Node nR = n.unsharedRight(); - - if ((nL == null || nR == null) && n.vOpt == null) { - if (attemptUnlink_nl(nParent, n)) { - // attempt to fix nParent.height while we've still got the lock - return fixHeight_nl(nParent); - } else { - // retry needed for n - return n; - } - } - - final int hN = n.height; - final int hL0 = height(nL); - final int hR0 = height(nR); - final int hNRepl = 1 + Math.max(hL0, hR0); - final int bal = hL0 - hR0; - - if (bal > 1) { - return rebalanceToRight_nl(nParent, n, nL, hR0); - } else if (bal < -1) { - return rebalanceToLeft_nl(nParent, n, nR, hL0); - } else if (hNRepl != hN) { - // we've got more than enough locks to do a height change, no need to - // trigger a retry - n.height = hNRepl; - - // nParent is already locked, let's try to fix it too - return fixHeight_nl(nParent); - } else { - // nothing to do - return null; - } - } - - private Node rebalanceToRight_nl(final Node nParent, - final Node n, - final Node nL, - final int hR0) { - // L is too large, we will rotate-right. If L.R is taller - // than L.L, then we will first rotate-left L. - synchronized (nL) { - final int hL = nL.height; - if (hL - hR0 <= 1) { - return n; // retry - } else { - final Node nLR = nL.unsharedRight(); - final int hLL0 = height(nL.left); - final int hLR0 = height(nLR); - if (hLL0 >= hLR0) { - // rotate right based on our snapshot of hLR - return rotateRight_nl(nParent, n, nL, hR0, hLL0, nLR, hLR0); - } else { - synchronized (nLR) { - // If our hLR snapshot is incorrect then we might - // actually need to do a single rotate-right on n. - final int hLR = nLR.height; - if (hLL0 >= hLR) { - return rotateRight_nl(nParent, n, nL, hR0, hLL0, nLR, hLR); - } else { - // If the underlying left balance would not be - // sufficient to actually fix n.left, then instead - // of rolling it into a double rotation we do it on - // it's own. This may let us avoid rotating n at - // all, but more importantly it avoids the creation - // of damaged nodes that don't have a direct - // ancestry relationship. The recursive call to - // rebalanceToRight_nl in this case occurs after we - // release the lock on nLR. - // - // We also need to avoid damaging n.left if post- - // rotation it would be an unnecessary routing node. - // Note that although our height snapshots might be - // stale, their zero/non-zero state can't be. - final int hLRL = height(nLR.left); - final int b = hLL0 - hLRL; - if (b >= -1 && b <= 1 && !((hLL0 == 0 || hLRL == 0) && nL.vOpt == null)) { - // nParent.child.left won't be damaged after a double rotation - return rotateRightOverLeft_nl(nParent, n, nL, hR0, hLL0, nLR, hLRL); - } - } - } - // focus on nL, if necessary n will be balanced later - return rebalanceToLeft_nl(n, nL, nLR, hLL0); - } - } - } - } - - private Node rebalanceToLeft_nl(final Node nParent, - final Node n, - final Node nR, - final int hL0) { - synchronized (nR) { - final int hR = nR.height; - if (hL0 - hR >= -1) { - return n; // retry - } else { - final Node nRL = nR.unsharedLeft(); - final int hRL0 = height(nRL); - final int hRR0 = height(nR.right); - if (hRR0 >= hRL0) { - return rotateLeft_nl(nParent, n, hL0, nR, nRL, hRL0, hRR0); - } else { - synchronized (nRL) { - final int hRL = nRL.height; - if (hRR0 >= hRL) { - return rotateLeft_nl(nParent, n, hL0, nR, nRL, hRL, hRR0); - } else { - final int hRLR = height(nRL.right); - final int b = hRR0 - hRLR; - if (b >= -1 && b <= 1 && !((hRR0 == 0 || hRLR == 0) && nR.vOpt == null)) { - return rotateLeftOverRight_nl(nParent, n, hL0, nR, nRL, hRR0, hRLR); - } - } - } - return rebalanceToRight_nl(n, nR, nRL, hRR0); - } - } - } - } - - private Node rotateRight_nl(final Node nParent, - final Node n, - final Node nL, - final int hR, - final int hLL, - final Node nLR, - final int hLR) { - final long nodeOVL = n.shrinkOVL; - - final Node nPL = nParent.left; - - n.shrinkOVL = beginChange(nodeOVL); - - n.left = nLR; - if (nLR != null) { - nLR.parent = n; - } - - nL.right = n; - n.parent = nL; - - if (nPL == n) { - nParent.left = nL; - } else { - nParent.right = nL; - } - nL.parent = nParent; - - // fix up heights links - final int hNRepl = 1 + Math.max(hLR, hR); - n.height = hNRepl; - nL.height = 1 + Math.max(hLL, hNRepl); - - n.shrinkOVL = endChange(nodeOVL); - - // We have damaged nParent, n (now parent.child.right), and nL (now - // parent.child). n is the deepest. Perform as many fixes as we can - // with the locks we've got. - - // We've already fixed the height for n, but it might still be outside - // our allowable balance range. In that case a simple fixHeight_nl - // won't help. - final int balN = hLR - hR; - if (balN < -1 || balN > 1) { - // we need another rotation at n - return n; - } - - // we've fixed balance and height damage for n, now handle - // extra-routing node damage - if ((nLR == null || hR == 0) && n.vOpt == null) { - // we need to remove n and then repair - return n; - } - - // we've already fixed the height at nL, do we need a rotation here? - final int balL = hLL - hNRepl; - if (balL < -1 || balL > 1) { - return nL; - } - - // nL might also have routing node damage (if nL.left was null) - if (hLL == 0 && nL.vOpt == null) { - return nL; - } - - // try to fix the parent height while we've still got the lock - return fixHeight_nl(nParent); - } - - private Node rotateLeft_nl(final Node nParent, - final Node n, - final int hL, - final Node nR, - final Node nRL, - final int hRL, - final int hRR) { - final long nodeOVL = n.shrinkOVL; - - final Node nPL = nParent.left; - - n.shrinkOVL = beginChange(nodeOVL); - - // fix up n links, careful to be compatible with concurrent traversal for all but n - n.right = nRL; - if (nRL != null) { - nRL.parent = n; - } - - nR.left = n; - n.parent = nR; - - if (nPL == n) { - nParent.left = nR; - } else { - nParent.right = nR; - } - nR.parent = nParent; - - // fix up heights - final int hNRepl = 1 + Math.max(hL, hRL); - n.height = hNRepl; - nR.height = 1 + Math.max(hNRepl, hRR); - - n.shrinkOVL = endChange(nodeOVL); - - final int balN = hRL - hL; - if (balN < -1 || balN > 1) { - return n; - } - - if ((nRL == null || hL == 0) && n.vOpt == null) { - return n; - } - - final int balR = hRR - hNRepl; - if (balR < -1 || balR > 1) { - return nR; - } - - if (hRR == 0 && nR.vOpt == null) { - return nR; - } - - return fixHeight_nl(nParent); - } - - private Node rotateRightOverLeft_nl(final Node nParent, - final Node n, - final Node nL, - final int hR, - final int hLL, - final Node nLR, - final int hLRL) { - final long nodeOVL = n.shrinkOVL; - final long leftOVL = nL.shrinkOVL; - - final Node nPL = nParent.left; - final Node nLRL = nLR.unsharedLeft(); - final Node nLRR = nLR.unsharedRight(); - final int hLRR = height(nLRR); - - n.shrinkOVL = beginChange(nodeOVL); - nL.shrinkOVL = beginChange(leftOVL); - - // fix up n links, careful about the order! - n.left = nLRR; - if (nLRR != null) { - nLRR.parent = n; - } - - nL.right = nLRL; - if (nLRL != null) { - nLRL.parent = nL; - } - - nLR.left = nL; - nL.parent = nLR; - nLR.right = n; - n.parent = nLR; - - if (nPL == n) { - nParent.left = nLR; - } else { - nParent.right = nLR; - } - nLR.parent = nParent; - - // fix up heights - final int hNRepl = 1 + Math.max(hLRR, hR); - n.height = hNRepl; - final int hLRepl = 1 + Math.max(hLL, hLRL); - nL.height = hLRepl; - nLR.height = 1 + Math.max(hLRepl, hNRepl); - - n.shrinkOVL = endChange(nodeOVL); - nL.shrinkOVL = endChange(leftOVL); - - // caller should have performed only a single rotation if nL was going - // to end up damaged - assert(Math.abs(hLL - hLRL) <= 1); - assert(!((hLL == 0 || nLRL == null) && nL.vOpt == null)); - - // We have damaged nParent, nLR (now parent.child), and n (now - // parent.child.right). n is the deepest. Perform as many fixes as we - // can with the locks we've got. - - // We've already fixed the height for n, but it might still be outside - // our allowable balance range. In that case a simple fixHeight_nl - // won't help. - final int balN = hLRR - hR; - if (balN < -1 || balN > 1) { - // we need another rotation at n - return n; - } - - // n might also be damaged by being an unnecessary routing node - if ((nLRR == null || hR == 0) && n.vOpt == null) { - // repair involves splicing out n and maybe more rotations - return n; - } - - // we've already fixed the height at nLR, do we need a rotation here? - final int balLR = hLRepl - hNRepl; - if (balLR < -1 || balLR > 1) { - return nLR; - } - - // try to fix the parent height while we've still got the lock - return fixHeight_nl(nParent); - } - - private Node rotateLeftOverRight_nl(final Node nParent, - final Node n, - final int hL, - final Node nR, - final Node nRL, - final int hRR, - final int hRLR) { - final long nodeOVL = n.shrinkOVL; - final long rightOVL = nR.shrinkOVL; - - final Node nPL = nParent.left; - final Node nRLL = nRL.unsharedLeft(); - final Node nRLR = nRL.unsharedRight(); - final int hRLL = height(nRLL); - - n.shrinkOVL = beginChange(nodeOVL); - nR.shrinkOVL = beginChange(rightOVL); - - // fix up n links, careful about the order! - n.right = nRLL; - if (nRLL != null) { - nRLL.parent = n; - } - - nR.left = nRLR; - if (nRLR != null) { - nRLR.parent = nR; - } - - nRL.right = nR; - nR.parent = nRL; - nRL.left = n; - n.parent = nRL; - - if (nPL == n) { - nParent.left = nRL; - } else { - nParent.right = nRL; - } - nRL.parent = nParent; - - // fix up heights - final int hNRepl = 1 + Math.max(hL, hRLL); - n.height = hNRepl; - final int hRRepl = 1 + Math.max(hRLR, hRR); - nR.height = hRRepl; - nRL.height = 1 + Math.max(hNRepl, hRRepl); - - n.shrinkOVL = endChange(nodeOVL); - nR.shrinkOVL = endChange(rightOVL); - - assert(Math.abs(hRR - hRLR) <= 1); - - final int balN = hRLL - hL; - if (balN < -1 || balN > 1) { - return n; - } - if ((nRLL == null || hL == 0) && n.vOpt == null) { - return n; - } - final int balRL = hRRepl - hNRepl; - if (balRL < -1 || balRL > 1) { - return nRL; - } - return fixHeight_nl(nParent); - } - - //////////////// Map views - - @Override - public NavigableSet keySet() { - return navigableKeySet(); - } - - @Override - public Set> entrySet() { - return new EntrySet(); - } - - private class EntrySet extends AbstractSet> { - - @Override - public int size() { - return SnapTreeMap.this.size(); - } - - @Override - public boolean isEmpty() { - return SnapTreeMap.this.isEmpty(); - } - - @Override - public void clear() { - SnapTreeMap.this.clear(); - } - - @Override - public boolean contains(final Object o) { - if (!(o instanceof Map.Entry)) { - return false; - } - final Object k = ((Map.Entry)o).getKey(); - final Object v = ((Map.Entry)o).getValue(); - final Object actualVo = SnapTreeMap.this.getImpl(k); - if (actualVo == null) { - // no associated value - return false; - } - final V actual = decodeNull(actualVo); - return v == null ? actual == null : v.equals(actual); - } - - @Override - public boolean add(final Entry e) { - final Object v = encodeNull(e.getValue()); - return update(e.getKey(), UpdateAlways, null, v) != v; - } - - @Override - public boolean remove(final Object o) { - if (!(o instanceof Map.Entry)) { - return false; - } - final Object k = ((Map.Entry)o).getKey(); - final Object v = ((Map.Entry)o).getValue(); - return SnapTreeMap.this.remove(k, v); - } - - @Override - public Iterator> iterator() { - return new EntryIter(SnapTreeMap.this); - } - } - - private static class EntryIter extends AbstractIter implements Iterator> { - private EntryIter(final SnapTreeMap m) { - super(m); - } - - private EntryIter(final SnapTreeMap m, - final Comparable minCmp, - final boolean minIncl, - final Comparable maxCmp, - final boolean maxIncl, - final boolean descending) { - super(m, minCmp, minIncl, maxCmp, maxIncl, descending); - } - - @Override - public Entry next() { - return nextNode(); - } - } - - private static class KeyIter extends AbstractIter implements Iterator { - private KeyIter(final SnapTreeMap m) { - super(m); - } - - private KeyIter(final SnapTreeMap m, - final Comparable minCmp, - final boolean minIncl, - final Comparable maxCmp, - final boolean maxIncl, - final boolean descending) { - super(m, minCmp, minIncl, maxCmp, maxIncl, descending); - } - - @Override - public K next() { - return nextNode().key; - } - } - - private static class AbstractIter { - private final SnapTreeMap m; - private final boolean descending; - private final char forward; - private final char reverse; - private Node[] path; - private int depth = 0; - private Node mostRecentNode; - private final K endKey; - - @SuppressWarnings("unchecked") - AbstractIter(final SnapTreeMap m) { - this.m = m; - this.descending = false; - this.forward = Right; - this.reverse = Left; - final Node root = m.holderRef.frozen().right; - this.path = (Node[]) new Node[1 + height(root)]; - this.endKey = null; - pushFirst(root); - } - - @SuppressWarnings("unchecked") - AbstractIter(final SnapTreeMap m, - final Comparable minCmp, - final boolean minIncl, - final Comparable maxCmp, - final boolean maxIncl, - final boolean descending) { - this.m = m; - this.descending = descending; - this.forward = !descending ? Right : Left; - this.reverse = !descending ? Left : Right; - final Comparable fromCmp; - final boolean fromIncl = !descending ? minIncl : maxIncl; - final Comparable toCmp; - final boolean toIncl = !descending ? maxIncl : minIncl; - if (!descending) { - fromCmp = minCmp; - toCmp = maxCmp; - } else { - fromCmp = maxCmp; - toCmp = minCmp; - } - - final Node root = m.holderRef.frozen().right; - - if (toCmp != null) { - this.endKey = (K) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, true, forward); - if (this.endKey == null) { - // no node satisfies the bound, nothing to iterate - // ---------> EARLY EXIT - return; - } - } else { - this.endKey = null; - } - - this.path = (Node[]) new Node[1 + height(root)]; - - if (fromCmp == null) { - pushFirst(root); - } - else { - pushFirst(root, fromCmp, fromIncl); - if (depth > 0 && top().vOpt == null) { - advance(); - } - } - } - - private int cmp(final Comparable comparable, final K key) { - final int c = comparable.compareTo(key); - if (!descending) { - return c; - } else { - return c == Integer.MIN_VALUE ? 1 : -c; - } - } - - private void pushFirst(Node node) { - while (node != null) { - path(node); - node = node.child(reverse); - } - } - - private void path(Node node) { - if (depth == path.length) - path = Arrays.copyOf(path, depth + 2); - - path[depth++] = node; - } - - private void pushFirst(Node node, final Comparable fromCmp, final boolean fromIncl) { - while (node != null) { - final int c = cmp(fromCmp, node.key); - if (c > 0 || (c == 0 && !fromIncl)) { - // everything we're interested in is on the right - node = node.child(forward); - } - else { - path(node); - if (c == 0) { - // start the iteration here - return; - } - else { - node = node.child(reverse); - } - } - } - } - - private Node top() { - return path[depth - 1]; - } - - private void advance() { - do { - final Node t = top(); - if (endKey != null && endKey == t.key) { - depth = 0; - path = null; - return; - } - - final Node fwd = t.child(forward); - if (fwd != null) { - pushFirst(fwd); - } else { - // keep going up until we pop a node that is a left child - Node popped; - do { - popped = path[--depth]; - } while (depth > 0 && popped == top().child(forward)); - } - - if (depth == 0) { - // clear out the path so we don't pin too much stuff - path = null; - return; - } - - // skip removed-but-not-unlinked entries - } while (top().vOpt == null); - } - - public boolean hasNext() { - return depth > 0; - } - - Node nextNode() { - if (depth == 0) { - throw new NoSuchElementException(); - } - mostRecentNode = top(); - advance(); - return mostRecentNode; - } - - public void remove() { - if (mostRecentNode == null) { - throw new IllegalStateException(); - } - m.remove(mostRecentNode.key); - mostRecentNode = null; - } - } - - //////////////// navigable keySet - - @Override - public NavigableSet navigableKeySet() { - return new KeySet(this) { - public Iterator iterator() { - return new KeyIter(SnapTreeMap.this); - } - }; - } - - @Override - public NavigableSet descendingKeySet() { - return descendingMap().navigableKeySet(); - } - - private abstract static class KeySet extends AbstractSet implements NavigableSet { - - private final ConcurrentNavigableMap map; - - protected KeySet(final ConcurrentNavigableMap map) { - this.map = map; - } - - //////// basic Set stuff - - @Override - abstract public Iterator iterator(); - - @Override - public boolean contains(final Object o) { return map.containsKey(o); } - @Override - public boolean isEmpty() { return map.isEmpty(); } - @Override - public int size() { return map.size(); } - @Override - public boolean remove(final Object o) { return map.remove(o) != null; } - - //////// SortedSet stuff - - @Override - public Comparator comparator() { return map.comparator(); } - @Override - public K first() { return map.firstKey(); } - @Override - public K last() { return map.lastKey(); } - - //////// NavigableSet stuff - - @Override - public K lower(final K k) { return map.lowerKey(k); } - @Override - public K floor(final K k) { return map.floorKey(k); } - @Override - public K ceiling(final K k) { return map.ceilingKey(k); } - @Override - public K higher(final K k) { return map.higherKey(k); } - - @Override - public K pollFirst() { return map.pollFirstEntry().getKey(); } - @Override - public K pollLast() { return map.pollLastEntry().getKey(); } - - @Override - public NavigableSet descendingSet() { return map.descendingKeySet(); } - @Override - public Iterator descendingIterator() { return map.descendingKeySet().iterator(); } - - @Override - public NavigableSet subSet(final K fromElement, final boolean minInclusive, final K toElement, final boolean maxInclusive) { - return map.subMap(fromElement, minInclusive, toElement, maxInclusive).keySet(); - } - @Override - public NavigableSet headSet(final K toElement, final boolean inclusive) { - return map.headMap(toElement, inclusive).keySet(); - } - @Override - public NavigableSet tailSet(final K fromElement, final boolean inclusive) { - return map.tailMap(fromElement, inclusive).keySet(); - } - @Override - public SortedSet subSet(final K fromElement, final K toElement) { - return map.subMap(fromElement, toElement).keySet(); - } - @Override - public SortedSet headSet(final K toElement) { - return map.headMap(toElement).keySet(); - } - @Override - public SortedSet tailSet(final K fromElement) { - return map.tailMap(fromElement).keySet(); - } - } - - //////////////// NavigableMap views - - @Override - public ConcurrentNavigableMap subMap(final K fromKey, - final boolean fromInclusive, - final K toKey, - final boolean toInclusive) { - final Comparable fromCmp = comparable(fromKey); - if (fromCmp.compareTo(toKey) > 0) { - throw new IllegalArgumentException(); - } - return new SubMap(this, fromKey, fromCmp, fromInclusive, toKey, comparable(toKey), toInclusive, false); - } - - @Override - public ConcurrentNavigableMap headMap(final K toKey, final boolean inclusive) { - return new SubMap(this, null, null, false, toKey, comparable(toKey), inclusive, false); - } - - @Override - public ConcurrentNavigableMap tailMap(final K fromKey, final boolean inclusive) { - return new SubMap(this, fromKey, comparable(fromKey), inclusive, null, null, false, false); - } - - @Override - public ConcurrentNavigableMap subMap(final K fromKey, final K toKey) { - return subMap(fromKey, true, toKey, false); - } - - @Override - public ConcurrentNavigableMap headMap(final K toKey) { - return headMap(toKey, false); - } - - @Override - public ConcurrentNavigableMap tailMap(final K fromKey) { - return tailMap(fromKey, true); - } - - @Override - public ConcurrentNavigableMap descendingMap() { - return new SubMap(this, null, null, false, null, null, false, true); - } - - private static class SubMap extends AbstractMap implements ConcurrentNavigableMap, Serializable { - /** */ - private static final long serialVersionUID = 0L; - - private final SnapTreeMap m; - private final K minKey; - private transient Comparable minCmp; - private final boolean minIncl; - private final K maxKey; - private transient Comparable maxCmp; - private final boolean maxIncl; - private final boolean descending; - - private SubMap(final SnapTreeMap m, - final K minKey, - final Comparable minCmp, - final boolean minIncl, - final K maxKey, - final Comparable maxCmp, - final boolean maxIncl, - final boolean descending) { - this.m = m; - this.minKey = minKey; - this.minCmp = minCmp; - this.minIncl = minIncl; - this.maxKey = maxKey; - this.maxCmp = maxCmp; - this.maxIncl = maxIncl; - this.descending = descending; - } - - // TODO: clone - - private boolean tooLow(final K key) { - if (minCmp == null) { - return false; - } else { - final int c = minCmp.compareTo(key); - return c > 0 || (c == 0 && !minIncl); - } - } - - private boolean tooHigh(final K key) { - if (maxCmp == null) { - return false; - } else { - final int c = maxCmp.compareTo(key); - return c < 0 || (c == 0 && !maxIncl); - } - } - - private boolean inRange(final K key) { - return !tooLow(key) && !tooHigh(key); - } - - private void requireInRange(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!inRange(key)) { - throw new IllegalArgumentException(); - } - } - - private char minDir() { - return descending ? Right : Left; - } - - private char maxDir() { - return descending ? Left : Right; - } - - //////// AbstractMap - - @Override - public boolean isEmpty() { - return m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, true, Left) == null; - } - - @Override - public int size() { - final Node root = m.holderRef.frozen().right; - return Node.computeFrozenSize(root, minCmp, minIncl, maxCmp, maxIncl); - } - - @Override - @SuppressWarnings("unchecked") - public boolean containsKey(final Object key) { - if (key == null) { - throw new NullPointerException(); - } - final K k = (K) key; - return inRange(k) && m.containsKey(k); - } - - @Override - public boolean containsValue(final Object value) { - // apply the same null policy as the rest of the code, but fall - // back to the default implementation - encodeNull(value); - return super.containsValue(value); - } - - @Override - @SuppressWarnings("unchecked") - public V get(final Object key) { - if (key == null) { - throw new NullPointerException(); - } - final K k = (K) key; - return !inRange(k) ? null : m.get(k); - } - - @Override - public V put(final K key, final V value) { - requireInRange(key); - return m.put(key, value); - } - - @Override - @SuppressWarnings("unchecked") - public V remove(final Object key) { - if (key == null) { - throw new NullPointerException(); - } - return !inRange((K) key) ? null : m.remove(key); - } - - @Override - public Set> entrySet() { - return new EntrySubSet(); - } - - private class EntrySubSet extends AbstractSet> { - public int size() { - return SubMap.this.size(); - } - - @Override - public boolean isEmpty() { - return SubMap.this.isEmpty(); - } - - @SuppressWarnings("unchecked") - @Override - public boolean contains(final Object o) { - if (!(o instanceof Map.Entry)) { - return false; - } - final Object k = ((Map.Entry)o).getKey(); - if (!inRange((K) k)) { - return false; - } - final Object v = ((Map.Entry)o).getValue(); - final Object actualVo = m.getImpl(k); - if (actualVo == null) { - // no associated value - return false; - } - final V actual = m.decodeNull(actualVo); - return v == null ? actual == null : v.equals(actual); - } - - @Override - public boolean add(final Entry e) { - requireInRange(e.getKey()); - final Object v = encodeNull(e.getValue()); - return m.update(e.getKey(), UpdateAlways, null, v) != v; - } - - @Override - public boolean remove(final Object o) { - if (!(o instanceof Map.Entry)) { - return false; - } - final Object k = ((Map.Entry)o).getKey(); - final Object v = ((Map.Entry)o).getValue(); - return SubMap.this.remove(k, v); - } - - @Override - public Iterator> iterator() { - return new EntryIter(m, minCmp, minIncl, maxCmp, maxIncl, descending); - } - } - - //////// SortedMap - - @Override - public Comparator comparator() { - final Comparator fromM = m.comparator(); - if (descending) { - return Collections.reverseOrder(fromM); - } else { - return fromM; - } - } - - @Override - public K firstKey() { - return m.boundedExtremeKeyOrThrow(minCmp, minIncl, maxCmp, maxIncl, minDir()); - } - - @Override - public K lastKey() { - return m.boundedExtremeKeyOrThrow(minCmp, minIncl, maxCmp, maxIncl, maxDir()); - } - - //////// NavigableMap - - @SuppressWarnings("unchecked") - private K firstKeyOrNull() { - return (K) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, true, minDir()); - } - - @SuppressWarnings("unchecked") - private K lastKeyOrNull() { - return (K) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, true, maxDir()); - } - - @SuppressWarnings("unchecked") - private Entry firstEntryOrNull() { - return (Entry) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, false, minDir()); - } - - @SuppressWarnings("unchecked") - private Entry lastEntryOrNull() { - return (Entry) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, false, maxDir()); - } - - @Override - public Entry lowerEntry(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooLow(key) : tooHigh(key)) { - return null; - } - return ((!descending ? tooHigh(key) : tooLow(key)) - ? this : subMapInRange(null, false, key, false)).lastEntryOrNull(); - } - - @Override - public K lowerKey(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooLow(key) : tooHigh(key)) { - return null; - } - return ((!descending ? tooHigh(key) : tooLow(key)) - ? this : subMapInRange(null, false, key, false)).lastKeyOrNull(); - } - - @Override - public Entry floorEntry(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooLow(key) : tooHigh(key)) { - return null; - } - return ((!descending ? tooHigh(key) : tooLow(key)) - ? this : subMapInRange(null, false, key, true)).lastEntryOrNull(); - } - - @Override - public K floorKey(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooLow(key) : tooHigh(key)) { - return null; - } - return ((!descending ? tooHigh(key) : tooLow(key)) - ? this : subMapInRange(null, false, key, true)).lastKeyOrNull(); - } - - @Override - public Entry ceilingEntry(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooHigh(key) : tooLow(key)) { - return null; - } - return ((!descending ? tooLow(key) : tooHigh(key)) - ? this : subMapInRange(key, true, null, false)).firstEntryOrNull(); - } - - @Override - public K ceilingKey(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooHigh(key) : tooLow(key)) { - return null; - } - return ((!descending ? tooLow(key) : tooHigh(key)) - ? this : subMapInRange(key, true, null, false)).firstKeyOrNull(); - } - - @Override - public Entry higherEntry(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooHigh(key) : tooLow(key)) { - return null; - } - return ((!descending ? tooLow(key) : tooHigh(key)) - ? this : subMapInRange(key, false, null, false)).firstEntryOrNull(); - } - - @Override - public K higherKey(final K key) { - if (key == null) { - throw new NullPointerException(); - } - if (!descending ? tooHigh(key) : tooLow(key)) { - return null; - } - return ((!descending ? tooLow(key) : tooHigh(key)) - ? this : subMapInRange(key, false, null, false)).firstKeyOrNull(); - } - - @Override - @SuppressWarnings("unchecked") - public Entry firstEntry() { - return (Entry) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, false, minDir()); - } - - @Override - @SuppressWarnings("unchecked") - public Entry lastEntry() { - return (Entry) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, false, maxDir()); - } - - @Override - @SuppressWarnings("unchecked") - public Entry pollFirstEntry() { - while (true) { - final Entry snapshot = (Entry) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, false, minDir()); - if (snapshot == null || m.remove(snapshot.getKey(), snapshot.getValue())) { - return snapshot; - } - } - } - - @Override - @SuppressWarnings("unchecked") - public Entry pollLastEntry() { - while (true) { - final Entry snapshot = (Entry) m.boundedExtreme(minCmp, minIncl, maxCmp, maxIncl, false, maxDir()); - if (snapshot == null || m.remove(snapshot.getKey(), snapshot.getValue())) { - return snapshot; - } - } - } - - //////// ConcurrentMap - - @Override - public V putIfAbsent(final K key, final V value) { - requireInRange(key); - return m.putIfAbsent(key, value); - } - - @Override - @SuppressWarnings("unchecked") - public boolean remove(final Object key, final Object value) { - return inRange((K) key) && m.remove(key, value); - } - - @Override - public boolean replace(final K key, final V oldValue, final V newValue) { - requireInRange(key); - return m.replace(key, oldValue, newValue); - } - - @Override - public V replace(final K key, final V value) { - requireInRange(key); - return m.replace(key, value); - } - - //////// ConcurrentNavigableMap - - @Override - public SubMap subMap(final K fromKey, - final boolean fromInclusive, - final K toKey, - final boolean toInclusive) { - if (fromKey == null || toKey == null) { - throw new NullPointerException(); - } - return subMapImpl(fromKey, fromInclusive, toKey, toInclusive); - } - - @Override - public SubMap headMap(final K toKey, final boolean inclusive) { - if (toKey == null) { - throw new NullPointerException(); - } - return subMapImpl(null, false, toKey, inclusive); - } - - @Override - public SubMap tailMap(final K fromKey, final boolean inclusive) { - if (fromKey == null) { - throw new NullPointerException(); - } - return subMapImpl(fromKey, inclusive, null, false); - } - - @Override - public SubMap subMap(final K fromKey, final K toKey) { - return subMap(fromKey, true, toKey, false); - } - - @Override - public SubMap headMap(final K toKey) { - return headMap(toKey, false); - } - - @Override - public SubMap tailMap(final K fromKey) { - return tailMap(fromKey, true); - } - - private SubMap subMapImpl(final K fromKey, - final boolean fromIncl, - final K toKey, - final boolean toIncl) { - if (fromKey != null) { - requireInRange(fromKey); - } - if (toKey != null) { - requireInRange(toKey); - } - return subMapInRange(fromKey, fromIncl, toKey, toIncl); - } - - private SubMap subMapInRange(final K fromKey, - final boolean fromIncl, - final K toKey, - final boolean toIncl) { - final Comparable fromCmp = fromKey == null ? null : m.comparable(fromKey); - final Comparable toCmp = toKey == null ? null : m.comparable(toKey); - - if (fromKey != null && toKey != null) { - final int c = fromCmp.compareTo(toKey); - if ((!descending ? c > 0 : c < 0)) { - throw new IllegalArgumentException(); - } - } - - K minK = minKey; - Comparable minC = minCmp; - boolean minI = minIncl; - K maxK = maxKey; - Comparable maxC = maxCmp; - boolean maxI = maxIncl; - - if (fromKey != null) { - if (!descending) { - minK = fromKey; - minC = fromCmp; - minI = fromIncl; - } else { - maxK = fromKey; - maxC = fromCmp; - maxI = fromIncl; - } - } - if (toKey != null) { - if (!descending) { - maxK = toKey; - maxC = toCmp; - maxI = toIncl; - } else { - minK = toKey; - minC = toCmp; - minI = toIncl; - } - } - - return new SubMap(m, minK, minC, minI, maxK, maxC, maxI, descending); - } - - @Override - public SubMap descendingMap() { - return new SubMap(m, minKey, minCmp, minIncl, maxKey, maxCmp, maxIncl, !descending); - } - - @Override - public NavigableSet keySet() { - return navigableKeySet(); - } - - @Override - public NavigableSet navigableKeySet() { - return new KeySet(SubMap.this) { - public Iterator iterator() { - return new KeyIter(m, minCmp, minIncl, maxCmp, maxIncl, descending); - } - }; - } - - @Override - public NavigableSet descendingKeySet() { - return descendingMap().navigableKeySet(); - } - - //////// Serialization - - private void readObject(final ObjectInputStream xi) throws IOException, ClassNotFoundException { - xi.defaultReadObject(); - - minCmp = minKey == null ? null : m.comparable(minKey); - maxCmp = maxKey == null ? null : m.comparable(maxKey); - } - } - - //////// Serialization - - /** Saves the state of the SnapTreeMap to a stream. */ - @SuppressWarnings("unchecked") - private void writeObject(final ObjectOutputStream xo) throws IOException { - // this handles the comparator, and any subclass stuff - xo.defaultWriteObject(); - - // by cloning the COWMgr, we get a frozen tree plus the size - final COWMgr h = (COWMgr) holderRef.clone(); - - xo.writeInt(h.size()); - writeEntry(xo, h.frozen().right); - } - - private void writeEntry(final ObjectOutputStream xo, final Node node) throws IOException { - if (node != null) { - writeEntry(xo, node.left); - if (node.vOpt != null) { - xo.writeObject(node.key); - xo.writeObject(decodeNull(node.vOpt)); - } - writeEntry(xo, node.right); - } - } - - /** Reverses {@link #writeObject(ObjectOutputStream)}. */ - private void readObject(final ObjectInputStream xi) throws IOException, ClassNotFoundException { - xi.defaultReadObject(); - - final int size = xi.readInt(); - - // TODO: take advantage of the sort order - // for now we optimize only by bypassing the COWMgr - final RootHolder holder = new RootHolder(); - for (int i = 0; i < size; ++i) { - final K k = (K) xi.readObject(); - final V v = (V) xi.readObject(); - updateUnderRoot(k, comparable(k), UpdateAlways, null, encodeNull(v), holder); - } - - holderRef = new COWMgr(holder, size); - } -} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/package-info.java deleted file mode 100644 index 2d75a8cd8c2f8..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/snaptree/package-info.java +++ /dev/null @@ -1,22 +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. - */ - -/** - * - * Snaptree and related classes. - */ -package org.apache.ignite.internal.util.snaptree; \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java index 4d1504736ae91..bf726d5d894ed 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java @@ -45,7 +45,6 @@ import org.apache.ignite.thread.GridThreadTest; import org.apache.ignite.thread.IgniteThreadPoolSizeTest; import org.apache.ignite.util.GridConcurrentLinkedDequeMultiThreadedTest; -import org.apache.ignite.util.GridIndexFillTest; import org.apache.ignite.util.GridIntListSelfTest; import org.apache.ignite.util.GridLogThrottleTest; import org.apache.ignite.util.GridLongListSelfTest; @@ -106,7 +105,6 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridMessageCollectionTest.class); suite.addTestSuite(WorkersControlMXBeanTest.class); suite.addTestSuite(GridConcurrentLinkedDequeMultiThreadedTest.class); - suite.addTestSuite(GridIndexFillTest.class); suite.addTestSuite(GridLogThrottleTest.class); suite.addTestSuite(GridRandomSelfTest.class); suite.addTestSuite(GridSnapshotLockSelfTest.class); diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java deleted file mode 100644 index 63635bd654802..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java +++ /dev/null @@ -1,259 +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.util; - -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.util.GridConcurrentSkipListSet; -import org.apache.ignite.internal.util.snaptree.SnapTreeMap; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * TODO write doc - */ -public class GridIndexFillTest extends GridCommonAbstractTest { - /** */ - private CopyOnWriteArrayList idxs; - - /** */ - private ConcurrentHashMap keyLocks; - - @Override protected void beforeTest() throws Exception { - super.beforeTest(); - - idxs = new CopyOnWriteArrayList<>(); - - idxs.add(new Idx(true)); - - keyLocks = new ConcurrentHashMap<>(); - } - - /** - * @param k Key. - */ - private CountDownLatch lock(String op, Integer k) { -// U.debug(op + " lock: " + k); - CountDownLatch latch = new CountDownLatch(1); - - for(;;) { - CountDownLatch l = keyLocks.putIfAbsent(k, latch); - - if (l == null) - return latch; - - try { - l.await(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - } - - /** - * @param k Key. - */ - private void unlock(Integer k, CountDownLatch latch) { -// U.debug("unlock: " + k); - assertTrue(keyLocks.remove(k, latch)); - - latch.countDown(); - } - - private void put(Integer k, Long v) { - CountDownLatch l = lock("add", k); - - for (Idx idx : idxs) - idx.add(k, v); - - unlock(k, l); - } - - private void remove(Integer k) { - CountDownLatch l = lock("rm", k); - - try { - Long v = null; - - for (Idx idx : idxs) { - Long v2 = idx.remove(k, v); - - if (v2 == null) { - assert v == null; - - return; // Nothing to remove. - } - - if (v == null) - v = v2; - else - assert v.equals(v2); - } - } - finally { - unlock(k, l); - } - } - - public void testSnaptreeParallelBuild() throws Exception { - final AtomicBoolean stop = new AtomicBoolean(); - - IgniteInternalFuture fut = multithreadedAsync(new Callable() { - @Override public Object call() throws Exception { - int i = ThreadLocalRandom.current().nextInt(100); - - while (!stop.get()) { - int k = i++ % 100; - long v = i++ % 10; - - if (i++ % 2 == 0) - put(k, v); - else - remove(k); - } - - return null; - } - }, 12, "put"); - - Thread.sleep(500); - - Idx newIdx = new Idx(false); - - idxs.add(newIdx); - - SnapTreeMap snap = idxs.get(0).tree.clone(); - - for (Map.Entry entry : snap.entrySet()) - newIdx.addX(entry.getKey(), entry.getValue()); - - newIdx.finish(); - - stop.set(true); - - fut.get(); - - assertEquals(idxs.get(0).tree, idxs.get(1).tree); - } - - private static class Idx { - - static int z = 1; - - private final SnapTreeMap tree = new SnapTreeMap<>(); //new ConcurrentSkipListMap<>(); - - private volatile Rm rm; - - private final String name = "idx" + z++; - - public Idx(boolean pk) { - if (!pk) - rm = new Rm(); - } - - public void add(Integer k, Long v) { -// U.debug(name + " add: k" + k + " " + v); - - Long old = tree.put(k, v); - - if (old != null) { - Rm rm = this.rm; - - if (rm != null) - rm.keys.add(k); - } - } - - public void addX(Integer k, Long v) { -// U.debug(name + " addX: k" + k + " " + v); - - assert v != null; - assert k != null; - -// Lock l = rm.lock.writeLock(); - -// l.lock(); - - try { - if (!rm.keys.contains(k)) { -// U.debug(name + " addX-put: k" + k + " " + v); - - tree.putIfAbsent(k, v); - } - } - finally { -// l.unlock(); - } - } - - public Long remove(Integer k, Long v) { - Rm rm = this.rm; - - if (rm != null) { - assert v != null; - -// Lock l = rm.lock.readLock(); - -// l.lock(); - - try { - rm.keys.add(k); - - Long v2 = tree.remove(k); - -// U.debug(name + " rm1: k" + k + " " + v + " " + v2); - - } - finally { -// l.unlock(); - } - } - else { - Long v2 = tree.remove(k); - -// U.debug(name + " rm2: k" + k + " " + v + " " + v2); - - if (v == null) - v = v2; - else - assertEquals(v, v2); - } - - return v; - } - - public void finish() { -// assertTrue(rm.tree.isEmpty()); - - rm = null; - } - } - - private static class Rm { -// private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - - private final GridConcurrentSkipListSet keys = new GridConcurrentSkipListSet<>(); - //new SnapTreeMap<>(); //new ConcurrentSkipListMap<>(); - } -} diff --git a/modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTestEntity.java b/modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTestEntity.java deleted file mode 100644 index 015ec96642a43..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTestEntity.java +++ /dev/null @@ -1,67 +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.loadtests.h2indexing; - -import java.util.Date; -import org.apache.ignite.cache.query.annotations.QuerySqlField; - -/** - * Test entity. - */ -public class GridTestEntity { - /** */ - @QuerySqlField(index = true) - private final String name; - - /** */ - @QuerySqlField(index = false) - private final Date date; - - /** - * Constructor. - * - * @param name Name. - * @param date Date. - */ - @SuppressWarnings("AssignmentToDateFieldFromParameter") - public GridTestEntity(String name, Date date) { - this.name = name; - this.date = date; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - GridTestEntity that = (GridTestEntity) o; - - return !(date != null ? !date.equals(that.date) : that.date != null) && - !(name != null ? !name.equals(that.name) : that.name != null); - - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = name != null ? name.hashCode() : 0; - - res = 31 * res + (date != null ? date.hashCode() : 0); - - return res; - } -} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTreeBenchmark.java b/modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTreeBenchmark.java deleted file mode 100644 index a711468484a64..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/loadtests/h2indexing/GridTreeBenchmark.java +++ /dev/null @@ -1,280 +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.loadtests.h2indexing; - -import java.util.Iterator; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.ignite.internal.util.snaptree.SnapTreeMap; - -/** - * NavigableMaps PUT benchmark. - */ -public class GridTreeBenchmark { - /** */ - private static final int PUTS = 8000000; - - /** */ - private static final int THREADS = 8; - - /** */ - private static final int ITERATIONS = PUTS / THREADS; - - /** - * Main method. - * - * @param args Command line args (not used). - * @throws BrokenBarrierException If failed. - * @throws InterruptedException If failed. - */ - public static void main(String... args) throws BrokenBarrierException, InterruptedException { - doTestMaps(); - } - - /** - * @throws BrokenBarrierException If failed. - * @throws InterruptedException If failed. - */ - private static void doTestAtomicInt() throws BrokenBarrierException, InterruptedException { - final AtomicInteger[] cnts = new AtomicInteger[8]; - - for (int i = 0; i < cnts.length; i++) - cnts[i] = new AtomicInteger(); - - final Thread[] ths = new Thread[THREADS]; - - final CyclicBarrier barrier = new CyclicBarrier(THREADS + 1); - - final AtomicInteger cnt = new AtomicInteger(); - - for (int i = 0; i < ths.length; i++) { - ths[i] = new Thread(new Runnable() { - @Override public void run() { - int idx = cnt.getAndIncrement(); - - AtomicInteger x = cnts[idx % cnts.length]; - - try { - barrier.await(); - } - catch (Exception e) { - throw new IllegalStateException(e); - } - - for (int i = 0; i < ITERATIONS; i++) - x.incrementAndGet(); - } - }); - - ths[i].start(); - } - - barrier.await(); - - long start = System.currentTimeMillis(); - - for (Thread t : ths) - t.join(); - - long time = System.currentTimeMillis() - start; - - System.out.println(time); - - } - - /** - * @throws BrokenBarrierException If failed. - * @throws InterruptedException If failed. - */ - private static void doTestMaps() throws BrokenBarrierException, InterruptedException { - final UUID[] data = generate(); - - @SuppressWarnings("unchecked") - final Map[] maps = new Map[4]; - - for (int i = 0; i < maps.length; i++) - maps[i] = - new SnapTreeMap<>(); - - - final Thread[] ths = new Thread[THREADS]; - - final CyclicBarrier barrier = new CyclicBarrier(THREADS + 1); - - final AtomicInteger cnt = new AtomicInteger(); - - for (int i = 0; i < ths.length; i++) { - ths[i] = new Thread(new Runnable() { - @Override public void run() { - int idx = cnt.getAndIncrement(); - - int off = idx * ITERATIONS; - - Map map = maps[idx % maps.length]; - - try { - barrier.await(); - } - catch (Exception e) { - throw new IllegalStateException(e); - } - - for (int i = 0; i < ITERATIONS; i++) { - UUID id = data[off + i]; - - id = map.put(id, id); - - assert id == null; - } - } - }); - - ths[i].start(); - } - - System.out.println("Sleep"); - Thread.sleep(10000); - - System.out.println("Go"); - barrier.await(); - - long start = System.currentTimeMillis(); - - for (Thread t : ths) - t.join(); - - long time = System.currentTimeMillis() - start; - - System.out.println(time); - } - - /** - * @throws BrokenBarrierException If failed. - * @throws InterruptedException If failed. - */ - private static void doBenchmark() throws BrokenBarrierException, InterruptedException { - int attemts = 20; - int warmups = 10; - - long snapTreeTime = 0; - long skipListTime = 0; - - for (int i = 0; i < attemts; i++) { - ConcurrentNavigableMap skipList = new ConcurrentSkipListMap<>(); - ConcurrentNavigableMap snapTree = new SnapTreeMap<>(); - - UUID[] ids = generate(); - - boolean warmup = i < warmups; - - snapTreeTime += doTest(snapTree, ids, warmup); - skipListTime += doTest(skipList, ids, warmup); - - assert skipList.size() == snapTree.size(); - - Iterator snapIt = snapTree.keySet().iterator(); - Iterator listIt = skipList.keySet().iterator(); - - for (int x = 0, len = skipList.size(); x < len; x++) - assert snapIt.next() == listIt.next(); - - System.out.println(i + " =================="); - } - - attemts -= warmups; - - System.out.println("Avg for GridSnapTreeMap: " + (snapTreeTime / attemts) + " ms"); - System.out.println("Avg for ConcurrentSkipListMap: " + (skipListTime / attemts) + " ms"); - } - - /** - * @return UUIDs. - */ - private static UUID[] generate() { - UUID[] ids = new UUID[ITERATIONS * THREADS]; - - for (int i = 0; i < ids.length; i++) - ids[i] = UUID.randomUUID(); - - return ids; - } - - /** - * @param tree Tree. - * @param data Data. - * @param warmup Warmup. - * @return Time. - * @throws BrokenBarrierException If failed. - * @throws InterruptedException If failed. - */ - private static long doTest(final ConcurrentNavigableMap tree, final UUID[] data, boolean warmup) - throws BrokenBarrierException, InterruptedException { - Thread[] ths = new Thread[THREADS]; - - final CyclicBarrier barrier = new CyclicBarrier(THREADS + 1); - - final AtomicInteger cnt = new AtomicInteger(); - - for (int i = 0; i < ths.length; i++) { - ths[i] = new Thread(new Runnable() { - @Override public void run() { - int off = cnt.getAndIncrement() * ITERATIONS; - - try { - barrier.await(); - } - catch (Exception e) { - throw new IllegalStateException(e); - } - - for (int i = 0; i < ITERATIONS; i++) { - UUID id = data[off + i]; - - id = tree.put(id, id); - - assert id == null; - } - } - }); - - ths[i].start(); - } - - barrier.await(); - - long start = System.currentTimeMillis(); - - for (Thread t : ths) - t.join(); - - long time = System.currentTimeMillis() - start; - - if (!warmup) { - System.out.println(tree.getClass().getSimpleName() + " " + time + " ms"); - - return time; - } - - return 0; - } -} From b119cf7e4136c2adccb54c704c7022a6209785dd Mon Sep 17 00:00:00 2001 From: shroman Date: Wed, 5 Sep 2018 16:24:07 +0900 Subject: [PATCH 48/95] IGNITE-9408: Update Apache Mesos version. - Fixes #4650. Signed-off-by: shroman --- modules/mesos/pom.xml | 2 +- .../java/org/apache/ignite/mesos/IgniteFramework.java | 3 +-- .../apache/ignite/mesos/IgniteSchedulerSelfTest.java | 11 +++++++++++ 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml index 58802599b735d..a8c7d8b2a2006 100644 --- a/modules/mesos/pom.xml +++ b/modules/mesos/pom.xml @@ -35,7 +35,7 @@ http://ignite.apache.org - 0.22.0 + 1.5.0 http://ignite.run/download_ignite.php https://archive.apache.org/dist/ignite/%s/apache-ignite-fabric-%s-bin.zip diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java index eea510a5379ec..addd3e0b017ea 100644 --- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java +++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java @@ -17,7 +17,6 @@ package org.apache.ignite.mesos; -import com.google.protobuf.ByteString; import java.util.logging.Level; import java.util.logging.Logger; import org.apache.ignite.mesos.resource.IgniteProvider; @@ -105,7 +104,7 @@ public static void main(String[] args) throws Exception { Protos.Credential cred = Protos.Credential.newBuilder() .setPrincipal(System.getenv(DEFAULT_PRINCIPAL)) - .setSecret(ByteString.copyFrom(System.getenv(DEFAULT_SECRET).getBytes())) + .setSecret(System.getenv(DEFAULT_SECRET)) .build(); driver = new MesosSchedulerDriver(scheduler, igniteFramework.getFrameworkInfo(), clusterProps.masterUrl(), diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java index 099daa2dd39cf..4e485698aa4f7 100644 --- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java +++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java @@ -465,6 +465,12 @@ public void clear() { return null; } + /** {@inheritDoc} */ + @Override public Protos.Status acceptOffers(Collection collection, + Collection collection1, Protos.Filters filters) { + return null; + } + /** {@inheritDoc} */ @Override public Protos.Status declineOffer(Protos.OfferID offerId, Protos.Filters filters) { declinedOffer = offerId; @@ -484,6 +490,11 @@ public void clear() { return null; } + /** {@inheritDoc} */ + @Override public Protos.Status suppressOffers() { + return null; + } + /** {@inheritDoc} */ @Override public Protos.Status acknowledgeStatusUpdate(Protos.TaskStatus status) { return null; From 5a2167b294b62b4ae0eda67ec5e3ddf19fd576ed Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 5 Sep 2018 11:29:01 +0300 Subject: [PATCH 49/95] IGNITE-6454 muted flaky tests - Fixes #4682. Signed-off-by: Dmitriy Govorukhin --- ...ePartitionedDataStructuresFailoverSelfTest.java | 14 ++++++++++++++ ...heReplicatedDataStructuresFailoverSelfTest.java | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedDataStructuresFailoverSelfTest.java index ecb2df9b7b1c9..eecfefe949816 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedDataStructuresFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedDataStructuresFailoverSelfTest.java @@ -38,4 +38,18 @@ public class GridCachePartitionedDataStructuresFailoverSelfTest @Override protected CacheAtomicityMode collectionCacheAtomicityMode() { return TRANSACTIONAL; } + + /** + * + */ + @Override public void testReentrantLockConstantTopologyChangeNonFailoverSafe() { + fail("https://issues.apache.org/jira/browse/IGNITE-6454"); + } + + /** + * + */ + @Override public void testFairReentrantLockConstantTopologyChangeNonFailoverSafe() { + fail("https://issues.apache.org/jira/browse/IGNITE-6454"); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java index cb6e77e8a2e96..27fbdcf1d48d7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedDataStructuresFailoverSelfTest.java @@ -52,4 +52,18 @@ public class GridCacheReplicatedDataStructuresFailoverSelfTest @Override public void testReentrantLockConstantMultipleTopologyChangeNonFailoverSafe() { fail("https://issues.apache.org/jira/browse/IGNITE-6454"); } + + /** + * + */ + @Override public void testReentrantLockConstantTopologyChangeNonFailoverSafe() { + fail("https://issues.apache.org/jira/browse/IGNITE-6454"); + } + + /** + * + */ + @Override public void testFairReentrantLockConstantTopologyChangeNonFailoverSafe() { + fail("https://issues.apache.org/jira/browse/IGNITE-6454"); + } } From d546e850d5ee375d061465d543608578e457f0bc Mon Sep 17 00:00:00 2001 From: Sergey Antonov Date: Wed, 5 Sep 2018 11:33:07 +0300 Subject: [PATCH 50/95] IGNITE-9438 Fix file descriptors leak in StandaloneWalRecordsIterator. - Fixes #4658. Signed-off-by: Dmitriy Govorukhin --- .../wal/AbstractWalRecordsIterator.java | 139 +++++++---- .../reader/StandaloneWalRecordsIterator.java | 33 +-- .../ignite/internal/util/IgniteUtils.java | 16 ++ .../StandaloneWalRecordsIteratorTest.java | 216 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 8 +- 5 files changed, 349 insertions(+), 63 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java index 9fbb53566b904..0b704caecd30c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.SegmentHeader; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.typedef.P2; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,8 +44,8 @@ import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.readSegmentHeader; /** - * Iterator over WAL segments. This abstract class provides most functionality for reading records in log. - * Subclasses are to override segment switching functionality + * Iterator over WAL segments. This abstract class provides most functionality for reading records in log. Subclasses + * are to override segment switching functionality */ public abstract class AbstractWalRecordsIterator extends GridCloseableIteratorAdapter> implements WALIterator { @@ -52,14 +53,14 @@ public abstract class AbstractWalRecordsIterator private static final long serialVersionUID = 0L; /** - * Current record preloaded, to be returned on next()
      - * Normally this should be not null because advance() method should already prepare some value
      + * Current record preloaded, to be returned on next()
      Normally this should be not null because advance() method + * should already prepare some value
      */ protected IgniteBiTuple curRec; /** - * Current WAL segment absolute index.
      - * Determined as lowest number of file at start, is changed during advance segment + * Current WAL segment absolute index.
      Determined as lowest number of file at start, is changed during advance + * segment */ protected long curWalSegmIdx = -1; @@ -179,7 +180,6 @@ protected void advance() throws IgniteCheckedException { } /** - * * @param tailReachedException Tail reached exception. * @param currWalSegment Current WAL segment read handler. * @return If need to throw exception after validation. @@ -210,9 +210,8 @@ protected IgniteCheckedException validateTailReachedException( } /** - * Switches records iterator to the next WAL segment - * as result of this method, new reference to segment should be returned. - * Null for current handle means stop of iteration. + * Switches records iterator to the next WAL segment as result of this method, new reference to segment should be + * returned. Null for current handle means stop of iteration. * * @param curWalSegment current open WAL segment or null if there is no open segment yet * @return new WAL segment to read or null for stop iteration @@ -263,8 +262,8 @@ protected IgniteBiTuple advanceRecord( } /** - * Performs final conversions with record loaded from WAL. - * To be overridden by subclasses if any processing required. + * Performs final conversions with record loaded from WAL. To be overridden by subclasses if any processing + * required. * * @param rec record to post process. * @return post processed record. @@ -278,11 +277,11 @@ protected IgniteBiTuple advanceRecord( * * @param e problem from records reading * @param ptr file pointer was accessed - * - * @return {@code null} if the error was handled and we can go ahead, - * {@code IgniteCheckedException} if the error was not handled, and we should stop the iteration. + * @return {@code null} if the error was handled and we can go ahead, {@code IgniteCheckedException} if the error + * was not handled, and we should stop the iteration. */ - protected IgniteCheckedException handleRecordException(@NotNull final Exception e, @Nullable final FileWALPointer ptr) { + protected IgniteCheckedException handleRecordException(@NotNull final Exception e, + @Nullable final FileWALPointer ptr) { if (log.isInfoEnabled()) log.info("Stopping WAL iteration due to an exception: " + e.getMessage() + ", ptr=" + ptr); @@ -290,45 +289,92 @@ protected IgniteCheckedException handleRecordException(@NotNull final Exception } /** + * Assumes fileIO will be closed in this method in case of error occurred. + * * @param desc File descriptor. - * @param start Optional start pointer. Null means read from the beginning - * @return Initialized file handle. - * @throws FileNotFoundException If segment file is missing. + * @param start Optional start pointer. Null means read from the beginning. + * @param fileIO fileIO associated with file descriptor + * @param segmentHeader read segment header from fileIO + * @return Initialized file read header. * @throws IgniteCheckedException If initialized failed due to another unexpected error. */ protected AbstractReadFileHandle initReadHandle( @NotNull final AbstractFileDescriptor desc, - @Nullable final FileWALPointer start - ) throws IgniteCheckedException, FileNotFoundException { + @Nullable final FileWALPointer start, + @NotNull final FileIO fileIO, + @NotNull final SegmentHeader segmentHeader + ) throws IgniteCheckedException { try { - FileIO fileIO = desc.isCompressed() ? new UnzipFileIO(desc.file()) : ioFactory.create(desc.file()); + final boolean isCompacted = segmentHeader.isCompacted(); + + if (isCompacted) + serializerFactory.skipPositionCheck(true); + + FileInput in = new FileInput(fileIO, buf); + + if (start != null && desc.idx() == start.index()) { + if (isCompacted) { + if (start.fileOffset() != 0) + serializerFactory.recordDeserializeFilter(new StartSeekingFilter(start)); + } + else { + // Make sure we skip header with serializer version. + long startOff = Math.max(start.fileOffset(), fileIO.position()); + in.seek(startOff); + } + } + + int serVer = segmentHeader.getSerializerVersion(); + + return createReadFileHandle(fileIO, desc.idx(), serializerFactory.createSerializer(serVer), in); + } + catch (SegmentEofException | EOFException ignore) { try { - SegmentHeader segmentHeader = readSegmentHeader(fileIO, curWalSegmIdx); + fileIO.close(); + } + catch (IOException ce) { + throw new IgniteCheckedException(ce); + } - boolean isCompacted = segmentHeader.isCompacted(); + return null; + } + catch (IgniteCheckedException e) { + U.closeWithSuppressingException(fileIO, e); - if (isCompacted) - serializerFactory.skipPositionCheck(true); + throw e; + } + catch (IOException e) { + U.closeWithSuppressingException(fileIO, e); - FileInput in = new FileInput(fileIO, buf); + throw new IgniteCheckedException( + "Failed to initialize WAL segment after reading segment header: " + desc.file().getAbsolutePath(), e); + } + } - if (start != null && desc.idx() == start.index()) { - if (isCompacted) { - if (start.fileOffset() != 0) - serializerFactory.recordDeserializeFilter(new StartSeekingFilter(start)); - } - else { - // Make sure we skip header with serializer version. - long startOff = Math.max(start.fileOffset(), fileIO.position()); + /** + * Assumes file descriptor will be opened in this method. The caller of this method must be responsible for closing + * opened file descriptor File descriptor will be closed ONLY in case of error occurred. + * + * @param desc File descriptor. + * @param start Optional start pointer. Null means read from the beginning + * @return Initialized file read header. + * @throws FileNotFoundException If segment file is missing. + * @throws IgniteCheckedException If initialized failed due to another unexpected error. + */ + protected AbstractReadFileHandle initReadHandle( + @NotNull final AbstractFileDescriptor desc, + @Nullable final FileWALPointer start + ) throws IgniteCheckedException, FileNotFoundException { + FileIO fileIO = null; - in.seek(startOff); - } - } + try { + fileIO = desc.isCompressed() ? new UnzipFileIO(desc.file()) : ioFactory.create(desc.file()); - int serVer = segmentHeader.getSerializerVersion(); + SegmentHeader segmentHeader; - return createReadFileHandle(fileIO, desc.idx(), serializerFactory.createSerializer(serVer), in); + try { + segmentHeader = readSegmentHeader(fileIO, curWalSegmIdx); } catch (SegmentEofException | EOFException ignore) { try { @@ -341,20 +387,21 @@ protected AbstractReadFileHandle initReadHandle( return null; } catch (IOException | IgniteCheckedException e) { - try { - fileIO.close(); - } - catch (IOException ce) { - e.addSuppressed(ce); - } + U.closeWithSuppressingException(fileIO, e); throw e; } + + return initReadHandle(desc, start, fileIO, segmentHeader); } catch (FileNotFoundException e) { + U.closeQuiet(fileIO); + throw e; } catch (IOException e) { + U.closeQuiet(fileIO); + throw new IgniteCheckedException( "Failed to initialize WAL segment: " + desc.file().getAbsolutePath(), e); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java index 67297905c1a43..a22287758b95e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java @@ -48,8 +48,10 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.SegmentHeader; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.NotNull; @@ -58,8 +60,8 @@ import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.readSegmentHeader; /** - * WAL reader iterator, for creation in standalone WAL reader tool - * Operates over one directory, does not provide start and end boundaries + * WAL reader iterator, for creation in standalone WAL reader tool Operates over one directory, does not provide start + * and end boundaries */ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator { /** Record buffer size */ @@ -88,12 +90,13 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator { /** * Creates iterator in file-by-file iteration mode. Directory + * * @param log Logger. * @param sharedCtx Shared context. Cache processor is to be configured if Cache Object Key & Data Entry is * required. * @param ioFactory File I/O factory. - * @param keepBinary Keep binary. This flag disables converting of non primitive types - * (BinaryObjects will be used instead) + * @param keepBinary Keep binary. This flag disables converting of non primitive types (BinaryObjects will be used + * instead) * @param walFiles Wal files. */ StandaloneWalRecordsIterator( @@ -128,8 +131,8 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator { } /** - * For directory mode sets oldest file as initial segment, - * for file by file mode, converts all files to descriptors and gets oldest as initial. + * For directory mode sets oldest file as initial segment, for file by file mode, converts all files to descriptors + * and gets oldest as initial. * * @param walFiles files for file-by-file iteration mode */ @@ -232,7 +235,6 @@ private void init(List walFiles) { } /** - * * @param ptr WAL pointer. * @return {@code True} If pointer between low and high bounds. {@code False} if not. */ @@ -243,7 +245,6 @@ private boolean checkBounds(WALPointer ptr) { } /** - * * @param idx WAL segment index. * @return {@code True} If pointer between low and high bounds. {@code False} if not. */ @@ -258,18 +259,21 @@ private boolean checkBounds(long idx) { ) throws IgniteCheckedException, FileNotFoundException { AbstractFileDescriptor fd = desc; - + FileIO fileIO = null; + SegmentHeader segmentHeader; while (true) { try { - FileIO fileIO = fd.isCompressed() ? new UnzipFileIO(fd.file()) : ioFactory.create(fd.file()); + fileIO = fd.isCompressed() ? new UnzipFileIO(fd.file()) : ioFactory.create(fd.file()); - readSegmentHeader(fileIO, curWalSegmIdx); + segmentHeader = readSegmentHeader(fileIO, curWalSegmIdx); break; } catch (IOException | IgniteCheckedException e) { log.error("Failed to init segment curWalSegmIdx=" + curWalSegmIdx + ", curIdx=" + curIdx, e); + U.closeQuiet(fileIO); + curIdx++; if (curIdx >= walFileDescriptors.size()) @@ -279,13 +283,13 @@ private boolean checkBounds(long idx) { } } - return super.initReadHandle(fd, start); + return initReadHandle(fd, start, fileIO, segmentHeader); } /** {@inheritDoc} */ @NotNull @Override protected WALRecord postProcessRecord(@NotNull final WALRecord rec) { - GridKernalContext kernalCtx = sharedCtx.kernalContext(); - IgniteCacheObjectProcessor processor = kernalCtx.cacheObjects(); + GridKernalContext kernalCtx = sharedCtx.kernalContext(); + IgniteCacheObjectProcessor processor = kernalCtx.cacheObjects(); if (processor != null && rec.type() == RecordType.DATA_RECORD) { try { @@ -335,6 +339,7 @@ private boolean checkBounds(long idx) { /** * Converts entry or lazy data entry into unwrapped entry + * * @param processor cache object processor for de-serializing objects. * @param fakeCacheObjCtx cache object context for de-serializing binary and unwrapping objects. * @param dataEntry entry to process 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 a68fb6d0b4112..d2d2b892c21c3 100755 --- 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 @@ -4019,6 +4019,22 @@ public static void close(@Nullable AutoCloseable rsrc, @Nullable IgniteLogger lo } } + /** + * Closes given resource suppressing possible checked exception. + * + * @param rsrc Resource to close. If it's {@code null} - it's no-op. + * @param e Suppressor exception + */ + public static void closeWithSuppressingException(@Nullable AutoCloseable rsrc, @NotNull Exception e) { + if (rsrc != null) + try { + rsrc.close(); + } + catch (Exception suppressed) { + e.addSuppressed(suppressed); + } + } + /** * Quietly closes given resource ignoring possible checked exception. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java new file mode 100644 index 0000000000000..b6a04d0a1488e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java @@ -0,0 +1,216 @@ +/* + * 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.persistence.wal.reader; + +import java.io.File; +import java.io.IOException; +import java.nio.file.OpenOption; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.record.SnapshotRecord; +import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; +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.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory.IteratorParametersBuilder; + +/** + * The test check, that StandaloneWalRecordsIterator correctly close file descriptors associated with WAL files. + */ +public class StandaloneWalRecordsIteratorTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(name); + + cfg.setDataStorageConfiguration( + new DataStorageConfiguration(). + setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + ) + ).setDiscoverySpi( + new TcpDiscoverySpi() + .setIpFinder(IP_FINDER) + ); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * Check correct closing file descriptors. + * + * @throws Exception if test failed. + */ + public void testCorrectClosingFileDescriptors() throws Exception { + IgniteEx ig = (IgniteEx)startGrid(); + + String archiveWalDir = getArchiveWalDirPath(ig); + + ig.cluster().active(true); + + IgniteCacheDatabaseSharedManager sharedMgr = ig.context().cache().context().database(); + + IgniteWriteAheadLogManager walMgr = ig.context().cache().context().wal(); + + // Generate WAL segments for filling WAL archive folder. + for (int i = 0; i < 2 * ig.configuration().getDataStorageConfiguration().getWalSegments(); i++) { + sharedMgr.checkpointReadLock(); + + try { + walMgr.log(new SnapshotRecord(i, false)); + } + finally { + sharedMgr.checkpointReadUnlock(); + } + } + + stopGrid(); + + // Iterate by all archived WAL segments. + createWalIterator(archiveWalDir).forEach(x -> { + }); + + assertTrue("At least one WAL file must be opened!", CountedFileIO.getCountOpenedWalFiles() > 0); + + assertEquals("All WAL files must be closed!", CountedFileIO.getCountOpenedWalFiles(), CountedFileIO.getCountClosedWalFiles()); + } + + /** + * Creates WALIterator associated with files inside walDir. + * + * @param walDir - path to WAL directory. + * @return WALIterator associated with files inside walDir. + * @throws IgniteCheckedException if error occur. + */ + private WALIterator createWalIterator(String walDir) throws IgniteCheckedException { + IteratorParametersBuilder params = new IteratorParametersBuilder(); + + params.ioFactory(new CountedFileIOFactory()); + + return new IgniteWalIteratorFactory(log).iterator(params.filesOrDirs(walDir)); + } + + /** + * Evaluate path to directory with WAL archive. + * + * @param ignite instance of Ignite. + * @return path to directory with WAL archive. + * @throws IgniteCheckedException if error occur. + */ + private String getArchiveWalDirPath(Ignite ignite) throws IgniteCheckedException { + return U.resolveWorkDirectory( + U.defaultWorkDirectory(), + ignite.configuration().getDataStorageConfiguration().getWalArchivePath(), + false + ).getAbsolutePath(); + } + + /** + * + */ + private static class CountedFileIOFactory extends RandomAccessFileIOFactory { + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + return create(file, StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + return new CountedFileIO(file, modes); + } + } + + /** + * + */ + private static class CountedFileIO extends RandomAccessFileIO { + /** Wal open counter. */ + private static final AtomicInteger WAL_OPEN_COUNTER = new AtomicInteger(); + /** Wal close counter. */ + private static final AtomicInteger WAL_CLOSE_COUNTER = new AtomicInteger(); + + /** File name. */ + private final String fileName; + + /** */ + public CountedFileIO(File file, OpenOption... modes) throws IOException { + super(file, modes); + + fileName = file.getName(); + + if (FileWriteAheadLogManager.WAL_NAME_PATTERN.matcher(fileName).matches()) + WAL_OPEN_COUNTER.incrementAndGet(); + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + super.close(); + + if (FileWriteAheadLogManager.WAL_NAME_PATTERN.matcher(fileName).matches()) + WAL_CLOSE_COUNTER.incrementAndGet(); + } + + /** + * + * @return number of opened files. + */ + public static int getCountOpenedWalFiles() { return WAL_OPEN_COUNTER.get(); } + + /** + * + * @return number of closed files. + */ + public static int getCountClosedWalFiles() { return WAL_CLOSE_COUNTER.get(); } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java index 5eba5a4511fe2..956d25623c9f4 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java @@ -34,7 +34,6 @@ import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteOnlineNodeOutOfBaselineFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsRebalancingOnNotStableTopologyTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsReserveWalSegmentsTest; -import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTransactionsHangTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWholeClusterRestartTest; import org.apache.ignite.internal.processors.cache.persistence.db.SlowHistoricalRebalanceSmallHistoryTest; import org.apache.ignite.internal.processors.cache.persistence.db.checkpoint.IgniteCheckpointDirtyPagesForLowLoadTest; @@ -53,13 +52,13 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalIteratorExceptionDuringReadTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalIteratorSwitchSegmentTest; -import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoverySeveralRestartsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalCompactionTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalDeletionArchiveFsyncTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalDeletionArchiveLogOnlyTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteDataIntegrityTests; import org.apache.ignite.internal.processors.cache.persistence.db.wal.reader.IgniteWalReaderTest; +import org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneWalRecordsIteratorTest; /** * @@ -89,7 +88,8 @@ public static TestSuite suite() { } /** - * Fills {@code suite} with PDS test subset, which operates with real page store, but requires long time to execute. + * Fills {@code suite} with PDS test subset, which operates with real page store, but requires long time to + * execute. * * @param suite suite to add tests into. */ @@ -171,6 +171,8 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgniteNodeStoppedDuringDisableWALTest.class); + suite.addTestSuite(StandaloneWalRecordsIteratorTest.class); + //suite.addTestSuite(IgniteWalRecoverySeveralRestartsTest.class); suite.addTestSuite(IgniteRebalanceScheduleResendPartitionsTest.class); From 672be18c0fc35359ef70696f10f6ed640c653cb3 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Wed, 5 Sep 2018 14:43:47 +0300 Subject: [PATCH 51/95] IGNITE-9402 Throws exception if WALWriter failed during flush buffer. - Fixes #4640. Signed-off-by: Dmitriy Govorukhin --- .../wal/FileWriteAheadLogManager.java | 22 ++++++++++--------- .../IgnitePdsWithIndexingCoreTestSuite.java | 2 ++ 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java index eb5ae95dd31e5..634cab3af757c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java @@ -78,7 +78,6 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; -import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.pagemem.wal.WALIterator; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; @@ -90,12 +89,12 @@ import org.apache.ignite.internal.processors.cache.WalStateManager.WALDisableContext; import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; -import org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator.AbstractFileDescriptor; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32; import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; @@ -2682,7 +2681,7 @@ public void writeHeader() { * * @param ptr Pointer. */ - private void flushOrWait(FileWALPointer ptr) { + private void flushOrWait(FileWALPointer ptr) throws IgniteCheckedException { if (ptr != null) { // If requested obsolete file index, it must be already flushed by close. if (ptr.index() != idx) @@ -2695,7 +2694,7 @@ private void flushOrWait(FileWALPointer ptr) { /** * @param ptr Pointer. */ - private void flush(FileWALPointer ptr) { + private void flush(FileWALPointer ptr) throws IgniteCheckedException { if (ptr == null) { // Unconditional flush. walWriter.flushAll(); @@ -3275,8 +3274,6 @@ private class WALWriter extends GridWorker { /** {@inheritDoc} */ @Override protected void body() { - Throwable err = null; - try { while (!isCancelled()) { while (waiters.isEmpty()) { @@ -3399,21 +3396,21 @@ private void unparkWaiters(long pos) { /** * Forces all made changes to the file. */ - void force() { + void force() throws IgniteCheckedException { flushBuffer(FILE_FORCE); } /** * Closes file. */ - void close() { + void close() throws IgniteCheckedException { flushBuffer(FILE_CLOSE); } /** * Flushes all data from the buffer. */ - void flushAll() { + void flushAll() throws IgniteCheckedException { flushBuffer(UNCONDITIONAL_FLUSH); } @@ -3421,7 +3418,7 @@ void flushAll() { * @param expPos Expected position. */ @SuppressWarnings("ForLoopReplaceableByForEach") - void flushBuffer(long expPos) { + void flushBuffer(long expPos) throws IgniteCheckedException { if (mmap) return; @@ -3447,6 +3444,11 @@ void flushBuffer(long expPos) { if (val == Long.MIN_VALUE) { waiters.remove(t); + Throwable walWriterError = walWriter.err; + + if (walWriterError != null) + throw new IgniteCheckedException("Flush buffer failed.", walWriterError); + return; } else diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java index 491bab7dd65bd..2989ccdddc150 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java @@ -72,7 +72,9 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgnitePdsTxHistoricalRebalancingTest.class); suite.addTestSuite(IgniteWalRecoveryPPCTest.class); + suite.addTestSuite(IgnitePdsDiskErrorsRecoveringTest.class); + suite.addTestSuite(IgnitePdsCacheDestroyDuringCheckpointTest.class); suite.addTestSuite(IgnitePdsBinaryMetadataOnClusterRestartTest.class); From b1dc2af4ea3332a94db4a52714d0fdfdbbd9a6c8 Mon Sep 17 00:00:00 2001 From: zzzadruga Date: Wed, 5 Sep 2018 14:52:04 +0300 Subject: [PATCH 52/95] IGNITE-8158: Wrap the method afterTestsStopped() call with try/catch. - Fixes #4464. Signed-off-by: Nikolay Izhikov --- .../junits/GridAbstractTest.java | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 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 a1d6c2528771a..ee0dfa48fd330 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 @@ -1773,6 +1773,8 @@ protected static ClassLoader getExternalClassLoader() { finally { serializedObj.clear(); + Exception err = null; + if (isLastTest()) { info(">>> Stopping test class: " + testClassDescription() + " <<<"); @@ -1787,11 +1789,22 @@ protected static ClassLoader getExternalClassLoader() { // Set reset flags, so counters will be reset on the next setUp. counters.setReset(true); - afterTestsStopped(); + try { + afterTestsStopped(); + } + catch (Exception e) { + err = e; + } - if(isSafeTopology()) + if (isSafeTopology()) { stopAllGrids(false); + if (stopGridErr) { + err = new RuntimeException("Not all Ignite instances has been stopped. " + + "Please, see log for details.", err); + } + } + // Remove counters. tests.remove(getClass()); @@ -1808,8 +1821,8 @@ protected static ClassLoader getExternalClassLoader() { cleanReferences(); - if (isLastTest() && isSafeTopology() && stopGridErr) - throw new RuntimeException("Not all Ignite instances has been stopped. Please, see log for details."); + if (err != null) + throw err; } } From 1a6141551949a2a29505281cc42085deec0ae7c4 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Wed, 5 Sep 2018 17:53:28 +0300 Subject: [PATCH 53/95] Revert TOPIC_MONGO removal in "IGNITE-9361 Remove IgniteInternalCache.isMongo*Cache() and other remnants." This partially reverts commit de30a86e740067ed239602fb5dc28396144fd446. - Fixes #4687. Signed-off-by: Dmitriy Govorukhin --- .../src/main/java/org/apache/ignite/internal/GridTopic.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 5610b2a2d4d6b..98a4d8d7be1ca 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 @@ -81,6 +81,12 @@ public enum GridTopic { /** */ TOPIC_CONTINUOUS, + /** + * @deprecated Should be removed in Apache Ignite 3.0. + */ + @Deprecated + TOPIC_MONGO, + /** */ TOPIC_TIME_SYNC, From 380078a3b9b2393fe19424237530a5230bf8a754 Mon Sep 17 00:00:00 2001 From: Alexey Platonov Date: Thu, 6 Sep 2018 12:08:36 +0300 Subject: [PATCH 54/95] IGNITE-9412: [ML] GDB convergence by error support. this closes #4670 --- ...DBOnTreesClassificationTrainerExample.java | 10 +- .../GDBOnTreesRegressionTrainerExample.java | 4 +- .../boosting/GDBBinaryClassifierTrainer.java | 57 +++--- .../boosting/GDBLearningStrategy.java | 115 +++++++++-- .../boosting/GDBRegressionTrainer.java | 8 +- .../ml/composition/boosting/GDBTrainer.java | 122 ++++++++---- .../convergence/ConvergenceChecker.java | 140 +++++++++++++ .../ConvergenceCheckerFactory.java | 58 ++++++ .../mean/MeanAbsValueConvergenceChecker.java | 116 +++++++++++ ...MeanAbsValueConvergenceCheckerFactory.java | 47 +++++ .../convergence/mean/package-info.java | 22 +++ .../MedianOfMedianConvergenceChecker.java | 126 ++++++++++++ ...dianOfMedianConvergenceCheckerFactory.java | 47 +++++ .../convergence/median/package-info.java | 22 +++ .../boosting/convergence/package-info.java | 24 +++ .../simple/ConvergenceCheckerStub.java | 79 ++++++++ .../simple/ConvergenceCheckerStubFactory.java | 48 +++++ .../convergence/simple/package-info.java | 24 +++ .../LogLoss.java} | 25 +-- .../ml/composition/boosting/loss/Loss.java | 45 +++++ .../boosting/loss/SquaredError.java | 36 ++++ .../boosting/loss/package-info.java | 22 +++ .../WeightedPredictionsAggregator.java | 10 + .../FeatureMatrixWithLabelsOnHeapData.java | 57 ++++++ ...tureMatrixWithLabelsOnHeapDataBuilder.java | 76 +++++++ .../boosting/GDBOnTreesLearningStrategy.java | 24 ++- .../ignite/ml/tree/data/DecisionTreeData.java | 36 ++-- .../composition/boosting/GDBTrainerTest.java | 81 +++++++- .../convergence/ConvergenceCheckerTest.java | 82 ++++++++ .../MeanAbsValueConvergenceCheckerTest.java | 73 +++++++ .../MedianOfMedianConvergenceCheckerTest.java | 57 ++++++ .../environment/LearningEnvironmentTest.java | 187 +++--------------- .../ignite/ml/knn/ANNClassificationTest.java | 12 +- 33 files changed, 1586 insertions(+), 306 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceChecker.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceChecker.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceChecker.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStub.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStubFactory.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/package-info.java rename modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/{LossGradientPerPredictionFunctions.java => loss/LogLoss.java} (53%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/Loss.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/SquaredError.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapData.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapDataBuilder.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerTest.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java index 075eab2ea8be4..e092e5cc84330 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesClassificationTrainerExample.java @@ -22,9 +22,8 @@ import org.apache.ignite.Ignition; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.ModelsComposition; -import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.composition.boosting.convergence.mean.MeanAbsValueConvergenceCheckerFactory; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.trainers.DatasetTrainer; import org.apache.ignite.ml.tree.boosting.GDBBinaryClassifierOnTreesTrainer; @@ -59,10 +58,11 @@ public static void main(String... args) throws InterruptedException { IgniteCache trainingSet = fillTrainingData(ignite, trainingSetCfg); // Create regression trainer. - DatasetTrainer trainer = new GDBBinaryClassifierOnTreesTrainer(1.0, 300, 2, 0.); + DatasetTrainer trainer = new GDBBinaryClassifierOnTreesTrainer(1.0, 300, 2, 0.) + .withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.1)); // Train decision tree model. - Model mdl = trainer.fit( + ModelsComposition mdl = trainer.fit( ignite, trainingSet, (k, v) -> VectorUtils.of(v[0]), @@ -80,6 +80,8 @@ public static void main(String... args) throws InterruptedException { System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", predicted, Math.sin(x) < 0 ? 0.0 : 1.0); } + System.out.println(">>> ---------------------------------"); + System.out.println(">>> Count of trees = " + mdl.getModels().size()); System.out.println(">>> ---------------------------------"); System.out.println(">>> GDB classification trainer example completed."); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java index b2b08d097b73b..3662973a3c007 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/boosting/GDBOnTreesRegressionTrainerExample.java @@ -24,6 +24,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.convergence.mean.MeanAbsValueConvergenceCheckerFactory; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.trainers.DatasetTrainer; @@ -59,7 +60,8 @@ public static void main(String... args) throws InterruptedException { IgniteCache trainingSet = fillTrainingData(ignite, trainingSetCfg); // Create regression trainer. - DatasetTrainer trainer = new GDBRegressionOnTreesTrainer(1.0, 2000, 1, 0.); + DatasetTrainer trainer = new GDBRegressionOnTreesTrainer(1.0, 2000, 1, 0.) + .withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.001)); // Train decision tree model. Model mdl = trainer.fit( diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBBinaryClassifierTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBBinaryClassifierTrainer.java index 37015574d2d1a..f6ddfedcead0d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBBinaryClassifierTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBBinaryClassifierTrainer.java @@ -19,24 +19,23 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import java.util.Set; import java.util.stream.Collectors; -import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.ml.composition.boosting.loss.LogLoss; +import org.apache.ignite.ml.composition.boosting.loss.Loss; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.structures.LabeledVector; import org.apache.ignite.ml.structures.LabeledVectorSet; import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap; /** - * Trainer for binary classifier using Gradient Boosting. - * As preparing stage this algorithm learn labels in dataset and create mapping dataset labels to 0 and 1. - * This algorithm uses gradient of Logarithmic Loss metric [LogLoss] by default in each step of learning. + * Trainer for binary classifier using Gradient Boosting. As preparing stage this algorithm learn labels in dataset and + * create mapping dataset labels to 0 and 1. This algorithm uses gradient of Logarithmic Loss metric [LogLoss] by + * default in each step of learning. */ public abstract class GDBBinaryClassifierTrainer extends GDBTrainer { /** External representation of first class. */ @@ -51,9 +50,7 @@ public abstract class GDBBinaryClassifierTrainer extends GDBTrainer { * @param cntOfIterations Count of learning iterations. */ public GDBBinaryClassifierTrainer(double gradStepSize, Integer cntOfIterations) { - super(gradStepSize, - cntOfIterations, - LossGradientPerPredictionFunctions.LOG_LOSS); + super(gradStepSize, cntOfIterations, new LogLoss()); } /** @@ -61,35 +58,37 @@ public GDBBinaryClassifierTrainer(double gradStepSize, Integer cntOfIterations) * * @param gradStepSize Grad step size. * @param cntOfIterations Count of learning iterations. - * @param lossGradient Gradient of loss function. First argument is sample size, second argument is valid answer, third argument is current model prediction. + * @param loss Loss function. */ - public GDBBinaryClassifierTrainer(double gradStepSize, - Integer cntOfIterations, - IgniteTriFunction lossGradient) { - - super(gradStepSize, cntOfIterations, lossGradient); + public GDBBinaryClassifierTrainer(double gradStepSize, Integer cntOfIterations, Loss loss) { + super(gradStepSize, cntOfIterations, loss); } /** {@inheritDoc} */ - @Override protected void learnLabels(DatasetBuilder builder, IgniteBiFunction featureExtractor, + @Override protected boolean learnLabels(DatasetBuilder builder, + IgniteBiFunction featureExtractor, IgniteBiFunction lExtractor) { - List uniqLabels = new ArrayList( - builder.build(new EmptyContextBuilder<>(), new LabeledDatasetPartitionDataBuilderOnHeap<>(featureExtractor, lExtractor)) - .compute((IgniteFunction, Set>) x -> + Set uniqLabels = builder.build(new EmptyContextBuilder<>(), new LabeledDatasetPartitionDataBuilderOnHeap<>(featureExtractor, lExtractor)) + .compute((IgniteFunction, Set>)x -> Arrays.stream(x.labels()).boxed().collect(Collectors.toSet()), (a, b) -> { - if (a == null) - return b; - if (b == null) - return a; - a.addAll(b); + if (a == null) + return b; + if (b == null) return a; - } - )); + a.addAll(b); + return a; + } + ); - A.ensure(uniqLabels.size() == 2, "Binary classifier expects two types of labels in learning dataset"); - externalFirstCls = uniqLabels.get(0); - externalSecondCls = uniqLabels.get(1); + if (uniqLabels != null && uniqLabels.size() == 2) { + ArrayList lblsArray = new ArrayList<>(uniqLabels); + externalFirstCls = lblsArray.get(0); + externalSecondCls = lblsArray.get(1); + return true; + } else { + return false; + } } /** {@inheritDoc} */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBLearningStrategy.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBLearningStrategy.java index 375748af1e792..737495e2df204 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBLearningStrategy.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBLearningStrategy.java @@ -22,6 +22,10 @@ import java.util.List; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.convergence.mean.MeanAbsValueConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.loss.Loss; import org.apache.ignite.ml.composition.predictionsaggregator.WeightedPredictionsAggregator; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.environment.LearningEnvironment; @@ -29,9 +33,9 @@ import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.functions.IgniteFunction; import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.trainers.DatasetTrainer; +import org.jetbrains.annotations.NotNull; /** * Learning strategy for gradient boosting. @@ -44,7 +48,7 @@ public class GDBLearningStrategy { protected int cntOfIterations; /** Loss of gradient. */ - protected IgniteTriFunction lossGradient; + protected Loss loss; /** External label to internal mapping. */ protected IgniteFunction externalLbToInternalMapping; @@ -61,9 +65,15 @@ public class GDBLearningStrategy { /** Composition weights. */ protected double[] compositionWeights; + /** Check convergence strategy factory. */ + protected ConvergenceCheckerFactory checkConvergenceStgyFactory = new MeanAbsValueConvergenceCheckerFactory(0.001); + + /** Default gradient step size. */ + private double defaultGradStepSize; + /** - * Implementation of gradient boosting iterations. At each step of iterations this algorithm - * build a regression model based on gradient of loss-function for current models composition. + * Implementation of gradient boosting iterations. At each step of iterations this algorithm build a regression + * model based on gradient of loss-function for current models composition. * * @param datasetBuilder Dataset builder. * @param featureExtractor Feature extractor. @@ -73,18 +83,43 @@ public class GDBLearningStrategy { public List> learnModels(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - List> models = new ArrayList<>(); + return update(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** + * Gets state of model in arguments, compare it with training parameters of trainer and if they are fit then + * trainer updates model in according to new data and return new model. In other case trains new model. + * + * @param mdlToUpdate Learned model. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + * @return Updated models list. + */ + public List> update(GDBTrainer.GDBModel mdlToUpdate, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + + List> models = initLearningState(mdlToUpdate); + + ConvergenceChecker convCheck = checkConvergenceStgyFactory.create(sampleSize, + externalLbToInternalMapping, loss, datasetBuilder, featureExtractor, lbExtractor); + DatasetTrainer, Double> trainer = baseMdlTrainerBuilder.get(); for (int i = 0; i < cntOfIterations; i++) { - double[] weights = Arrays.copyOf(compositionWeights, i); + double[] weights = Arrays.copyOf(compositionWeights, models.size()); WeightedPredictionsAggregator aggregator = new WeightedPredictionsAggregator(weights, meanLabelValue); - Model currComposition = new ModelsComposition(models, aggregator); + ModelsComposition currComposition = new ModelsComposition(models, aggregator); + if (convCheck.isConverged(datasetBuilder, currComposition)) + break; IgniteBiFunction lbExtractorWrap = (k, v) -> { Double realAnswer = externalLbToInternalMapping.apply(lbExtractor.apply(k, v)); Double mdlAnswer = currComposition.apply(featureExtractor.apply(k, v)); - return -lossGradient.apply(sampleSize, realAnswer, mdlAnswer); + return -loss.gradient(sampleSize, realAnswer, mdlAnswer); }; long startTs = System.currentTimeMillis(); @@ -96,6 +131,29 @@ public List> learnModels(DatasetBuilder datas return models; } + /** + * Restores state of already learned model if can and sets learning parameters according to this state. + * + * @param mdlToUpdate Model to update. + * @return list of already learned models. + */ + @NotNull protected List> initLearningState(GDBTrainer.GDBModel mdlToUpdate) { + List> models = new ArrayList<>(); + if(mdlToUpdate != null) { + models.addAll(mdlToUpdate.getModels()); + WeightedPredictionsAggregator aggregator = (WeightedPredictionsAggregator) mdlToUpdate.getPredictionsAggregator(); + meanLabelValue = aggregator.getBias(); + compositionWeights = new double[models.size() + cntOfIterations]; + for(int i = 0; i < models.size(); i++) + compositionWeights[i] = aggregator.getWeights()[i]; + } else { + compositionWeights = new double[cntOfIterations]; + } + + Arrays.fill(compositionWeights, models.size(), compositionWeights.length, defaultGradStepSize); + return models; + } + /** * Sets learning environment. * @@ -117,12 +175,12 @@ public GDBLearningStrategy withCntOfIterations(int cntOfIterations) { } /** - * Sets gradient of loss function. + * Loss function. * - * @param lossGradient Loss gradient. + * @param loss Loss function. */ - public GDBLearningStrategy withLossGradient(IgniteTriFunction lossGradient) { - this.lossGradient = lossGradient; + public GDBLearningStrategy withLossGradient(Loss loss) { + this.loss = loss; return this; } @@ -141,7 +199,8 @@ public GDBLearningStrategy withExternalLabelToInternal(IgniteFunction, Double>> buildBaseMdlTrainer) { + public GDBLearningStrategy withBaseModelTrainerBuilder( + IgniteSupplier, Double>> buildBaseMdlTrainer) { this.baseMdlTrainerBuilder = buildBaseMdlTrainer; return this; } @@ -175,4 +234,34 @@ public GDBLearningStrategy withCompositionWeights(double[] compositionWeights) { this.compositionWeights = compositionWeights; return this; } + + /** + * Sets CheckConvergenceStgyFactory. + * + * @param factory Factory. + */ + public GDBLearningStrategy withCheckConvergenceStgyFactory(ConvergenceCheckerFactory factory) { + this.checkConvergenceStgyFactory = factory; + return this; + } + + /** + * Sets default gradient step size. + * + * @param defaultGradStepSize Default gradient step size. + */ + public GDBLearningStrategy withDefaultGradStepSize(double defaultGradStepSize) { + this.defaultGradStepSize = defaultGradStepSize; + return this; + } + + /** */ + public double[] getCompositionWeights() { + return compositionWeights; + } + + /** */ + public double getMeanValue() { + return meanLabelValue; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBRegressionTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBRegressionTrainer.java index 201586ee1c72e..8c1afd7283b76 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBRegressionTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBRegressionTrainer.java @@ -17,6 +17,7 @@ package org.apache.ignite.ml.composition.boosting; +import org.apache.ignite.ml.composition.boosting.loss.SquaredError; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.primitives.vector.Vector; @@ -33,15 +34,14 @@ public abstract class GDBRegressionTrainer extends GDBTrainer { * @param cntOfIterations Count of learning iterations. */ public GDBRegressionTrainer(double gradStepSize, Integer cntOfIterations) { - super(gradStepSize, - cntOfIterations, - LossGradientPerPredictionFunctions.MSE); + super(gradStepSize, cntOfIterations, new SquaredError()); } /** {@inheritDoc} */ - @Override protected void learnLabels(DatasetBuilder builder, IgniteBiFunction featureExtractor, + @Override protected boolean learnLabels(DatasetBuilder builder, IgniteBiFunction featureExtractor, IgniteBiFunction lExtractor) { + return true; } /** {@inheritDoc} */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java index c7f21dd1fa362..85af7983e2a0c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/GDBTrainer.java @@ -22,6 +22,9 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.convergence.mean.MeanAbsValueConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.loss.Loss; import org.apache.ignite.ml.composition.predictionsaggregator.WeightedPredictionsAggregator; import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; @@ -30,7 +33,7 @@ import org.apache.ignite.ml.environment.logging.MLLogger; import org.apache.ignite.ml.knn.regression.KNNRegressionTrainer; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer; import org.apache.ignite.ml.regressions.linear.LinearRegressionSGDTrainer; @@ -60,24 +63,25 @@ public abstract class GDBTrainer extends DatasetTrainer lossGradient; + protected final Loss loss; + + /** Check convergence strategy factory. */ + protected ConvergenceCheckerFactory checkConvergenceStgyFactory = new MeanAbsValueConvergenceCheckerFactory(0.001); /** * Constructs GDBTrainer instance. * * @param gradStepSize Grad step size. * @param cntOfIterations Count of learning iterations. - * @param lossGradient Gradient of loss function. First argument is sample size, second argument is valid answer + * @param loss Gradient of loss function. First argument is sample size, second argument is valid answer * third argument is current model prediction. */ - public GDBTrainer(double gradStepSize, Integer cntOfIterations, - IgniteTriFunction lossGradient) { + public GDBTrainer(double gradStepSize, Integer cntOfIterations, Loss loss) { gradientStep = gradStepSize; this.cntOfIterations = cntOfIterations; - this.lossGradient = lossGradient; + this.loss = loss; } /** {@inheritDoc} */ @@ -85,53 +89,55 @@ public GDBTrainer(double gradStepSize, Integer cntOfIterations, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - learnLabels(datasetBuilder, featureExtractor, lbExtractor); + return updateModel(null, datasetBuilder, featureExtractor, lbExtractor); + } + + /** {@inheritDoc} */ + @Override protected ModelsComposition updateModel(ModelsComposition mdl, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + if (!learnLabels(datasetBuilder, featureExtractor, lbExtractor)) + return getLastTrainedModelOrThrowEmptyDatasetException(mdl); + + IgniteBiTuple initAndSampleSize = computeInitialValue(datasetBuilder, featureExtractor, lbExtractor); + if(initAndSampleSize == null) + return getLastTrainedModelOrThrowEmptyDatasetException(mdl); - IgniteBiTuple initAndSampleSize = computeInitialValue(datasetBuilder, - featureExtractor, lbExtractor); Double mean = initAndSampleSize.get1(); Long sampleSize = initAndSampleSize.get2(); - double[] compositionWeights = new double[cntOfIterations]; - Arrays.fill(compositionWeights, gradientStep); - WeightedPredictionsAggregator resAggregator = new WeightedPredictionsAggregator(compositionWeights, mean); - long learningStartTs = System.currentTimeMillis(); - List> models = getLearningStrategy() + GDBLearningStrategy stgy = getLearningStrategy() .withBaseModelTrainerBuilder(this::buildBaseModelTrainer) .withExternalLabelToInternal(this::externalLabelToInternal) .withCntOfIterations(cntOfIterations) - .withCompositionWeights(compositionWeights) .withEnvironment(environment) - .withLossGradient(lossGradient) + .withLossGradient(loss) .withSampleSize(sampleSize) .withMeanLabelValue(mean) - .learnModels(datasetBuilder, featureExtractor, lbExtractor); + .withDefaultGradStepSize(gradientStep) + .withCheckConvergenceStgyFactory(checkConvergenceStgyFactory); + + List> models; + if (mdl != null) + models = stgy.update((GDBModel)mdl, datasetBuilder, featureExtractor, lbExtractor); + else + models = stgy.learnModels(datasetBuilder, featureExtractor, lbExtractor); double learningTime = (double)(System.currentTimeMillis() - learningStartTs) / 1000.0; environment.logger(getClass()).log(MLLogger.VerboseLevel.LOW, "The training time was %.2fs", learningTime); - return new ModelsComposition(models, resAggregator) { - @Override public Double apply(Vector features) { - return internalLabelToExternal(super.apply(features)); - } - }; - } - - - //TODO: This method will be implemented in IGNITE-9412 - /** {@inheritDoc} */ - @Override public ModelsComposition updateModel(ModelsComposition mdl, DatasetBuilder datasetBuilder, - IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - - throw new UnsupportedOperationException(); + WeightedPredictionsAggregator resAggregator = new WeightedPredictionsAggregator( + stgy.getCompositionWeights(), + stgy.getMeanValue() + ); + return new GDBModel(models, resAggregator, this::internalLabelToExternal); } - //TODO: This method will be implemented in IGNITE-9412 /** {@inheritDoc} */ @Override protected boolean checkState(ModelsComposition mdl) { - throw new UnsupportedOperationException(); + return mdl instanceof GDBModel; } /** @@ -140,8 +146,9 @@ public GDBTrainer(double gradStepSize, Integer cntOfIterations, * @param builder Dataset builder. * @param featureExtractor Feature extractor. * @param lExtractor Labels extractor. + * @return true if labels learning was successful. */ - protected abstract void learnLabels(DatasetBuilder builder, + protected abstract boolean learnLabels(DatasetBuilder builder, IgniteBiFunction featureExtractor, IgniteBiFunction lExtractor); /** @@ -196,7 +203,8 @@ protected IgniteBiTuple computeInitialValue(DatasetBuilder< } ); - meanTuple.set1(meanTuple.get1() / meanTuple.get2()); + if (meanTuple != null) + meanTuple.set1(meanTuple.get1() / meanTuple.get2()); return meanTuple; } catch (Exception e) { @@ -204,6 +212,17 @@ protected IgniteBiTuple computeInitialValue(DatasetBuilder< } } + /** + * Sets CheckConvergenceStgyFactory. + * + * @param factory + * @return trainer. + */ + public GDBTrainer withCheckConvergenceStgyFactory(ConvergenceCheckerFactory factory) { + this.checkConvergenceStgyFactory = factory; + return this; + } + /** * Returns learning strategy. * @@ -212,4 +231,33 @@ protected IgniteBiTuple computeInitialValue(DatasetBuilder< protected GDBLearningStrategy getLearningStrategy() { return new GDBLearningStrategy(); } + + /** */ + public static class GDBModel extends ModelsComposition { + /** Serial version uid. */ + private static final long serialVersionUID = 3476661240155508004L; + + /** Internal to external lbl mapping. */ + private final IgniteFunction internalToExternalLblMapping; + + /** + * Creates an instance of GDBModel. + * + * @param models Models. + * @param predictionsAggregator Predictions aggregator. + * @param internalToExternalLblMapping Internal to external lbl mapping. + */ + public GDBModel(List> models, + WeightedPredictionsAggregator predictionsAggregator, + IgniteFunction internalToExternalLblMapping) { + + super(models, predictionsAggregator); + this.internalToExternalLblMapping = internalToExternalLblMapping; + } + + /** {@inheritDoc} */ + @Override public Double apply(Vector features) { + return internalToExternalLblMapping.apply(super.apply(features)); + } + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceChecker.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceChecker.java new file mode 100644 index 0000000000000..3f6e8cafa6447 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceChecker.java @@ -0,0 +1,140 @@ +/* + * 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.ml.composition.boosting.convergence; + +import java.io.Serializable; +import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapData; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapDataBuilder; +import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Contains logic of error computing and convergence checking for Gradient Boosting algorithms. + * + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + */ +public abstract class ConvergenceChecker implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 710762134746674105L; + + /** Sample size. */ + private long sampleSize; + + /** External label to internal mapping. */ + private IgniteFunction externalLbToInternalMapping; + + /** Loss function. */ + private Loss loss; + + /** Feature extractor. */ + private IgniteBiFunction featureExtractor; + + /** Label extractor. */ + private IgniteBiFunction lbExtractor; + + /** Precision of convergence check. */ + private double precision; + + /** + * Constructs an instance of ConvergenceChecker. + * + * @param sampleSize Sample size. + * @param externalLbToInternalMapping External label to internal mapping. + * @param loss Loss gradient. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @param precision + */ + public ConvergenceChecker(long sampleSize, + IgniteFunction externalLbToInternalMapping, Loss loss, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, + double precision) { + + assert precision < 1 && precision >= 0; + + this.sampleSize = sampleSize; + this.externalLbToInternalMapping = externalLbToInternalMapping; + this.loss = loss; + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + this.precision = precision; + } + + /** + * Checks convergency on dataset. + * + * @param currMdl Current model. + * @return true if GDB is converged. + */ + public boolean isConverged(DatasetBuilder datasetBuilder, ModelsComposition currMdl) { + try (Dataset dataset = datasetBuilder.build( + new EmptyContextBuilder<>(), + new FeatureMatrixWithLabelsOnHeapDataBuilder<>(featureExtractor, lbExtractor) + )) { + return isConverged(dataset, currMdl); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Checks convergency on dataset. + * + * @param dataset Dataset. + * @param currMdl Current model. + * @return true if GDB is converged. + */ + public boolean isConverged(Dataset dataset, ModelsComposition currMdl) { + Double error = computeMeanErrorOnDataset(dataset, currMdl); + return error < precision || error.isNaN(); + } + + /** + * Compute error for given model on learning dataset. + * + * @param dataset Learning dataset. + * @param mdl Model. + * @return error mean value. + */ + public abstract Double computeMeanErrorOnDataset( + Dataset dataset, + ModelsComposition mdl); + + /** + * Compute error for the specific vector of dataset. + * + * @param currMdl Current model. + * @return error. + */ + public double computeError(Vector features, Double answer, ModelsComposition currMdl) { + Double realAnswer = externalLbToInternalMapping.apply(answer); + Double mdlAnswer = currMdl.apply(features); + return -loss.gradient(sampleSize, realAnswer, mdlAnswer); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerFactory.java new file mode 100644 index 0000000000000..7592f509571ba --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerFactory.java @@ -0,0 +1,58 @@ +/* + * 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.ml.composition.boosting.convergence; + +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Factory for ConvergenceChecker. + */ +public abstract class ConvergenceCheckerFactory { + /** Precision of error checking. If error <= precision then it is equated to 0.0*/ + protected double precision; + + /** + * Creates an instance of ConvergenceCheckerFactory. + * + * @param precision Precision [0 <= precision < 1]. + */ + public ConvergenceCheckerFactory(double precision) { + this.precision = precision; + } + + /** + * Create an instance of ConvergenceChecker. + * + * @param sampleSize Sample size. + * @param externalLbToInternalMapping External label to internal mapping. + * @param loss Loss function. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @return ConvergenceCheckerFactory instance. + */ + public abstract ConvergenceChecker create(long sampleSize, + IgniteFunction externalLbToInternalMapping, Loss loss, + DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceChecker.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceChecker.java new file mode 100644 index 0000000000000..7340bfaa64d2b --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceChecker.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.ml.composition.boosting.convergence.mean; + +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapData; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; + +/** + * Use mean value of errors for estimating error on dataset. + * + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + */ +public class MeanAbsValueConvergenceChecker extends ConvergenceChecker { + /** Serial version uid. */ + private static final long serialVersionUID = 8534776439755210864L; + + /** + * Creates an intance of MeanAbsValueConvergenceChecker. + * + * @param sampleSize Sample size. + * @param externalLbToInternalMapping External label to internal mapping. + * @param loss Loss. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + */ + public MeanAbsValueConvergenceChecker(long sampleSize, IgniteFunction externalLbToInternalMapping, + Loss loss, DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor, + double precision) { + + super(sampleSize, externalLbToInternalMapping, loss, datasetBuilder, featureExtractor, lbExtractor, precision); + } + + /** {@inheritDoc} */ + @Override public Double computeMeanErrorOnDataset(Dataset dataset, + ModelsComposition mdl) { + + IgniteBiTuple sumAndCnt = dataset.compute( + partition -> computeStatisticOnPartition(mdl, partition), + this::reduce + ); + + if(sumAndCnt == null || sumAndCnt.getValue() == 0) + return Double.NaN; + return sumAndCnt.getKey() / sumAndCnt.getValue(); + } + + /** + * Compute sum of absolute value of errors and count of rows in partition. + * + * @param mdl Model. + * @param part Partition. + * @return Tuple (sum of errors, count of rows) + */ + private IgniteBiTuple computeStatisticOnPartition(ModelsComposition mdl, FeatureMatrixWithLabelsOnHeapData part) { + Double sum = 0.0; + + for(int i = 0; i < part.getFeatures().length; i++) { + double error = computeError(VectorUtils.of(part.getFeatures()[i]), part.getLabels()[i], mdl); + sum += Math.abs(error); + } + + return new IgniteBiTuple<>(sum, (long) part.getLabels().length); + } + + /** + * Merge left and right statistics from partitions. + * + * @param left Left. + * @param right Right. + * @return merged value. + */ + private IgniteBiTuple reduce(IgniteBiTuple left, IgniteBiTuple right) { + if (left == null) { + if (right != null) + return right; + else + return new IgniteBiTuple<>(0.0, 0L); + } + + if (right == null) + return left; + + return new IgniteBiTuple<>( + left.getKey() + right.getKey(), + right.getValue() + left.getValue() + ); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerFactory.java new file mode 100644 index 0000000000000..f02a60631f074 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerFactory.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.ml.composition.boosting.convergence.mean; + +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Factory for {@link MeanAbsValueConvergenceChecker}. + */ +public class MeanAbsValueConvergenceCheckerFactory extends ConvergenceCheckerFactory { + /** + * @param precision Precision. + */ + public MeanAbsValueConvergenceCheckerFactory(double precision) { + super(precision); + } + + /** {@inheritDoc} */ + @Override public ConvergenceChecker create(long sampleSize, + IgniteFunction externalLbToInternalMapping, Loss loss, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + return new MeanAbsValueConvergenceChecker<>(sampleSize, externalLbToInternalMapping, loss, + datasetBuilder, featureExtractor, lbExtractor, precision); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/package-info.java new file mode 100644 index 0000000000000..1ab6e66a66616 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/mean/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 implementation of convergence checking computer by mean of absolute value of errors in dataset. + */ +package org.apache.ignite.ml.composition.boosting.convergence.mean; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceChecker.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceChecker.java new file mode 100644 index 0000000000000..7e66a9c77f97c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceChecker.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.ml.composition.boosting.convergence.median; + +import java.util.Arrays; +import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapData; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; + +/** + * Use median of median on partitions value of errors for estimating error on dataset. This algorithm may be less + * sensitive to + * + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + */ +public class MedianOfMedianConvergenceChecker extends ConvergenceChecker { + /** Serial version uid. */ + private static final long serialVersionUID = 4902502002933415287L; + + /** + * Creates an instance of MedianOfMedianConvergenceChecker. + * + * @param sampleSize Sample size. + * @param lblMapping External label to internal mapping. + * @param loss Loss function. + * @param datasetBuilder Dataset builder. + * @param fExtr Feature extractor. + * @param lbExtr Label extractor. + * @param precision Precision. + */ + public MedianOfMedianConvergenceChecker(long sampleSize, IgniteFunction lblMapping, Loss loss, + DatasetBuilder datasetBuilder, IgniteBiFunction fExtr, + IgniteBiFunction lbExtr, double precision) { + + super(sampleSize, lblMapping, loss, datasetBuilder, fExtr, lbExtr, precision); + } + + /** {@inheritDoc} */ + @Override public Double computeMeanErrorOnDataset(Dataset dataset, + ModelsComposition mdl) { + + double[] medians = dataset.compute( + data -> computeMedian(mdl, data), + this::reduce + ); + + if(medians == null) + return Double.POSITIVE_INFINITY; + return getMedian(medians); + } + + /** + * Compute median value on data partition. + * + * @param mdl Model. + * @param data Data. + * @return median value. + */ + private double[] computeMedian(ModelsComposition mdl, FeatureMatrixWithLabelsOnHeapData data) { + double[] errors = new double[data.getLabels().length]; + for (int i = 0; i < errors.length; i++) + errors[i] = Math.abs(computeError(VectorUtils.of(data.getFeatures()[i]), data.getLabels()[i], mdl)); + return new double[] {getMedian(errors)}; + } + + /** + * Compute median value on array of errors. + * + * @param errors Error values. + * @return median value of errors. + */ + private double getMedian(double[] errors) { + if(errors.length == 0) + return Double.POSITIVE_INFINITY; + + Arrays.sort(errors); + final int middleIdx = (errors.length - 1) / 2; + if (errors.length % 2 == 1) + return errors[middleIdx]; + else + return (errors[middleIdx + 1] + errors[middleIdx]) / 2; + } + + /** + * Merge median values among partitions. + * + * @param left Left partition. + * @param right Right partition. + * @return merged median values. + */ + private double[] reduce(double[] left, double[] right) { + if (left == null) + return right; + if(right == null) + return left; + + double[] res = new double[left.length + right.length]; + System.arraycopy(left, 0, res, 0, left.length); + System.arraycopy(right, 0, res, left.length, right.length); + return res; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerFactory.java new file mode 100644 index 0000000000000..a1affe00b44e0 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerFactory.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.ml.composition.boosting.convergence.median; + +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Factory for {@link MedianOfMedianConvergenceChecker}. + */ +public class MedianOfMedianConvergenceCheckerFactory extends ConvergenceCheckerFactory { + /** + * @param precision Precision. + */ + public MedianOfMedianConvergenceCheckerFactory(double precision) { + super(precision); + } + + /** {@inheritDoc} */ + @Override public ConvergenceChecker create(long sampleSize, + IgniteFunction externalLbToInternalMapping, Loss loss, DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + return new MedianOfMedianConvergenceChecker<>(sampleSize, externalLbToInternalMapping, loss, + datasetBuilder, featureExtractor, lbExtractor, precision); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/package-info.java new file mode 100644 index 0000000000000..3798ef90f95c0 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/median/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 implementation of convergence checking computer by median of medians of errors in dataset. + */ +package org.apache.ignite.ml.composition.boosting.convergence.median; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/package-info.java new file mode 100644 index 0000000000000..6d42c627f37bc --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/package-info.java @@ -0,0 +1,24 @@ +/* + * 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 contains implementation of convergency checking algorithms for gradient boosting. + * This algorithms may stop training of gradient boosting if it achieve error on dataset less than precision + * specified by user. + */ +package org.apache.ignite.ml.composition.boosting.convergence; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStub.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStub.java new file mode 100644 index 0000000000000..716d04ef88af9 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStub.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.ml.composition.boosting.convergence.simple; + +import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapData; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * This strategy skip estimating error on dataset step. + * According to this strategy, training will stop after reaching the maximum number of iterations. + * + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + */ +public class ConvergenceCheckerStub extends ConvergenceChecker { + /** Serial version uid. */ + private static final long serialVersionUID = 8534776439755210864L; + + /** + * Creates an intance of ConvergenceCheckerStub. + * + * @param sampleSize Sample size. + * @param externalLbToInternalMapping External label to internal mapping. + * @param loss Loss function. + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + */ + public ConvergenceCheckerStub(long sampleSize, + IgniteFunction externalLbToInternalMapping, Loss loss, + DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + + super(sampleSize, externalLbToInternalMapping, loss, datasetBuilder, + featureExtractor, lbExtractor, 0.0); + } + + /** {@inheritDoc} */ + @Override public boolean isConverged(DatasetBuilder datasetBuilder, ModelsComposition currMdl) { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean isConverged(Dataset dataset, + ModelsComposition currMdl) { + return false; + } + + /** {@inheritDoc} */ + @Override public Double computeMeanErrorOnDataset(Dataset dataset, + ModelsComposition mdl) { + + throw new UnsupportedOperationException(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStubFactory.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStubFactory.java new file mode 100644 index 0000000000000..a0f0d5ced3b20 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/ConvergenceCheckerStubFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.composition.boosting.convergence.simple; + +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * Factory for {@link ConvergenceCheckerStub}. + */ +public class ConvergenceCheckerStubFactory extends ConvergenceCheckerFactory { + /** + * Create an instance of ConvergenceCheckerStubFactory. + */ + public ConvergenceCheckerStubFactory() { + super(0.0); + } + + /** {@inheritDoc} */ + @Override public ConvergenceChecker create(long sampleSize, + IgniteFunction externalLbToInternalMapping, Loss loss, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + + return new ConvergenceCheckerStub<>(sampleSize, externalLbToInternalMapping, loss, + datasetBuilder, featureExtractor, lbExtractor); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/package-info.java new file mode 100644 index 0000000000000..915903ad69f33 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/convergence/simple/package-info.java @@ -0,0 +1,24 @@ +/* + * 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 implementation of Stub for convergence checking. + * By this implementation gradient boosting will train new submodels until count of models achieving max value [count + * of iterations parameter]. + */ +package org.apache.ignite.ml.composition.boosting.convergence.simple; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/LossGradientPerPredictionFunctions.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/LogLoss.java similarity index 53% rename from modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/LossGradientPerPredictionFunctions.java rename to modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/LogLoss.java index 488c0e35a7add..19ef70bb546cb 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/LossGradientPerPredictionFunctions.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/LogLoss.java @@ -15,19 +15,22 @@ * limitations under the License. */ -package org.apache.ignite.ml.composition.boosting; - -import org.apache.ignite.ml.math.functions.IgniteTriFunction; +package org.apache.ignite.ml.composition.boosting.loss; /** - * Contains implementations of per-prediction loss functions for gradient boosting algorithm. + * Logistic regression loss function. */ -public class LossGradientPerPredictionFunctions { - /** Mean squared error loss for regression. */ - public static IgniteTriFunction MSE = - (sampleSize, answer, prediction) -> (2.0 / sampleSize) * (prediction - answer); +public class LogLoss implements Loss { + /** Serial version uid. */ + private static final long serialVersionUID = 2251384437214194977L; + + /** {@inheritDoc} */ + @Override public double error(long sampleSize, double answer, double prediction) { + return -(answer * Math.log(prediction) + (1 - answer) * Math.log(1 - prediction)); + } - /** Logarithmic loss for binary classification. */ - public static IgniteTriFunction LOG_LOSS = - (sampleSize, answer, prediction) -> (prediction - answer) / (prediction * (1.0 - prediction)); + /** {@inheritDoc} */ + @Override public double gradient(long sampleSize, double answer, double prediction) { + return (prediction - answer) / (prediction * (1.0 - prediction)); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/Loss.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/Loss.java new file mode 100644 index 0000000000000..72fff30af1473 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/Loss.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.composition.boosting.loss; + +import java.io.Serializable; + +/** + * Loss interface of computing error or gradient of error on specific row in dataset. + */ +public interface Loss extends Serializable { + /** + * Error value for model answer. + * + * @param sampleSize Sample size. + * @param lb Label. + * @param mdlAnswer Model answer. + * @return error value. + */ + public double error(long sampleSize, double lb, double mdlAnswer); + + /** + * Error gradient value for model answer. + * + * @param sampleSize Sample size. + * @param lb Label. + * @param mdlAnswer Model answer. + * @return error value. + */ + public double gradient(long sampleSize, double lb, double mdlAnswer); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/SquaredError.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/SquaredError.java new file mode 100644 index 0000000000000..8f2f17e770712 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/SquaredError.java @@ -0,0 +1,36 @@ +/* + * 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.ml.composition.boosting.loss; + +/** + * Represent error function as E(label, modelAnswer) = 1/N * (label - prediction)^2 + */ +public class SquaredError implements Loss { + /** Serial version uid. */ + private static final long serialVersionUID = 564886150646352157L; + + /** {@inheritDoc} */ + @Override public double error(long sampleSize, double lb, double prediction) { + return Math.pow(lb - prediction, 2) / sampleSize; + } + + /** {@inheritDoc} */ + @Override public double gradient(long sampleSize, double lb, double prediction) { + return (2.0 / sampleSize) * (prediction - lb); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/package-info.java new file mode 100644 index 0000000000000..83a5e39d597d3 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/boosting/loss/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 loss functions for Gradient Boosting algorithms. + */ +package org.apache.ignite.ml.composition.boosting.loss; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java b/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java index 8a369ad1d0567..5e0f7f18ffd22 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/composition/predictionsaggregator/WeightedPredictionsAggregator.java @@ -86,4 +86,14 @@ public WeightedPredictionsAggregator(double[] weights, double bias) { return builder.append(bias > 0 ? " + " : " - ").append(String.format("%.4f", bias)) .append("]").toString(); } + + /** */ + public double[] getWeights() { + return weights; + } + + /** */ + public double getBias() { + return bias; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapData.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapData.java new file mode 100644 index 0000000000000..9dbc1a9cb38c3 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapData.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.ml.dataset.primitive; + +/** + * A partition {@code data} of the containing matrix of features and vector of labels stored in heap. + */ +public class FeatureMatrixWithLabelsOnHeapData implements AutoCloseable { + /** Matrix with features. */ + private final double[][] features; + + /** Vector with labels. */ + private final double[] labels; + + /** + * Constructs an instance of FeatureMatrixWithLabelsOnHeapData. + * + * @param features Features. + * @param labels Labels. + */ + public FeatureMatrixWithLabelsOnHeapData(double[][] features, double[] labels) { + assert features.length == labels.length : "Features and labels have to be the same length"; + + this.features = features; + this.labels = labels; + } + + /** */ + public double[][] getFeatures() { + return features; + } + + /** */ + public double[] getLabels() { + return labels; + } + + /** {@inheritDoc} */ + @Override public void close() { + // Do nothing, GC will clean up. + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapDataBuilder.java new file mode 100644 index 0000000000000..be1724c44a1ef --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/primitive/FeatureMatrixWithLabelsOnHeapDataBuilder.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.ml.dataset.primitive; + +import java.io.Serializable; +import java.util.Iterator; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.dataset.UpstreamEntry; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.tree.data.DecisionTreeData; + +/** + * A partition {@code data} builder that makes {@link DecisionTreeData}. + * + * @param Type of a key in upstream data. + * @param Type of a value in upstream data. + * @param Type of a partition context. + */ +public class FeatureMatrixWithLabelsOnHeapDataBuilder + implements PartitionDataBuilder { + /** Serial version uid. */ + private static final long serialVersionUID = 6273736987424171813L; + + /** Function that extracts features from an {@code upstream} data. */ + private final IgniteBiFunction featureExtractor; + + /** Function that extracts labels from an {@code upstream} data. */ + private final IgniteBiFunction lbExtractor; + + /** + * Constructs a new instance of decision tree data builder. + * + * @param featureExtractor Function that extracts features from an {@code upstream} data. + * @param lbExtractor Function that extracts labels from an {@code upstream} data. + */ + public FeatureMatrixWithLabelsOnHeapDataBuilder(IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + } + + /** {@inheritDoc} */ + @Override public FeatureMatrixWithLabelsOnHeapData build(Iterator> upstreamData, long upstreamDataSize, C ctx) { + double[][] features = new double[Math.toIntExact(upstreamDataSize)][]; + double[] labels = new double[Math.toIntExact(upstreamDataSize)]; + + int ptr = 0; + while (upstreamData.hasNext()) { + UpstreamEntry entry = upstreamData.next(); + + features[ptr] = featureExtractor.apply(entry.getKey(), entry.getValue()).asArray(); + + labels[ptr] = lbExtractor.apply(entry.getKey(), entry.getValue()); + + ptr++; + } + + return new FeatureMatrixWithLabelsOnHeapData(features, labels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBOnTreesLearningStrategy.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBOnTreesLearningStrategy.java index 8589a79529992..6ebbda14d2314 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBOnTreesLearningStrategy.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBOnTreesLearningStrategy.java @@ -17,12 +17,13 @@ package org.apache.ignite.ml.tree.boosting; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.composition.boosting.GDBLearningStrategy; +import org.apache.ignite.ml.composition.boosting.GDBTrainer; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; import org.apache.ignite.ml.composition.predictionsaggregator.WeightedPredictionsAggregator; import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; @@ -54,22 +55,30 @@ public GDBOnTreesLearningStrategy(boolean useIndex) { } /** {@inheritDoc} */ - @Override public List> learnModels(DatasetBuilder datasetBuilder, - IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + @Override public List> update(GDBTrainer.GDBModel mdlToUpdate, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { DatasetTrainer, Double> trainer = baseMdlTrainerBuilder.get(); assert trainer instanceof DecisionTree; DecisionTree decisionTreeTrainer = (DecisionTree) trainer; - List> models = new ArrayList<>(); + List> models = initLearningState(mdlToUpdate); + + ConvergenceChecker convCheck = checkConvergenceStgyFactory.create(sampleSize, + externalLbToInternalMapping, loss, datasetBuilder, featureExtractor, lbExtractor); + try (Dataset dataset = datasetBuilder.build( new EmptyContextBuilder<>(), new DecisionTreeDataBuilder<>(featureExtractor, lbExtractor, useIndex) )) { for (int i = 0; i < cntOfIterations; i++) { - double[] weights = Arrays.copyOf(compositionWeights, i); + double[] weights = Arrays.copyOf(compositionWeights, models.size()); WeightedPredictionsAggregator aggregator = new WeightedPredictionsAggregator(weights, meanLabelValue); - Model currComposition = new ModelsComposition(models, aggregator); + ModelsComposition currComposition = new ModelsComposition(models, aggregator); + + if(convCheck.isConverged(dataset, currComposition)) + break; dataset.compute(part -> { if(part.getCopyOfOriginalLabels() == null) @@ -78,7 +87,7 @@ public GDBOnTreesLearningStrategy(boolean useIndex) { for(int j = 0; j < part.getLabels().length; j++) { double mdlAnswer = currComposition.apply(VectorUtils.of(part.getFeatures()[j])); double originalLbVal = externalLbToInternalMapping.apply(part.getCopyOfOriginalLabels()[j]); - part.getLabels()[j] = -lossGradient.apply(sampleSize, originalLbVal, mdlAnswer); + part.getLabels()[j] = -loss.gradient(sampleSize, originalLbVal, mdlAnswer); } }); @@ -92,6 +101,7 @@ public GDBOnTreesLearningStrategy(boolean useIndex) { throw new RuntimeException(e); } + compositionWeights = Arrays.copyOf(compositionWeights, models.size()); return models; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeData.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeData.java index d5750ea57a50f..b8a16dc93e3da 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeData.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeData.java @@ -19,18 +19,14 @@ import java.util.ArrayList; import java.util.List; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapData; import org.apache.ignite.ml.tree.TreeFilter; /** - * A partition {@code data} of the containing matrix of features and vector of labels stored in heap. + * A partition {@code data} of the containing matrix of features and vector of labels stored in heap + * with index on features. */ -public class DecisionTreeData implements AutoCloseable { - /** Matrix with features. */ - private final double[][] features; - - /** Vector with labels. */ - private final double[] labels; - +public class DecisionTreeData extends FeatureMatrixWithLabelsOnHeapData implements AutoCloseable { /** Copy of vector with original labels. Auxiliary for Gradient Boosting on Trees.*/ private double[] copyOfOriginalLabels; @@ -48,10 +44,7 @@ public class DecisionTreeData implements AutoCloseable { * @param buildIdx Build index. */ public DecisionTreeData(double[][] features, double[] labels, boolean buildIdx) { - assert features.length == labels.length : "Features and labels have to be the same length"; - - this.features = features; - this.labels = labels; + super(features, labels); this.buildIndex = buildIdx; indexesCache = new ArrayList<>(); @@ -68,6 +61,8 @@ public DecisionTreeData(double[][] features, double[] labels, boolean buildIdx) public DecisionTreeData filter(TreeFilter filter) { int size = 0; + double[][] features = getFeatures(); + double[] labels = getLabels(); for (int i = 0; i < features.length; i++) if (filter.test(features[i])) size++; @@ -95,12 +90,15 @@ public DecisionTreeData filter(TreeFilter filter) { * @param col Column. */ public void sort(int col) { - sort(col, 0, features.length - 1); + sort(col, 0, getFeatures().length - 1); } /** */ private void sort(int col, int from, int to) { if (from < to) { + double[][] features = getFeatures(); + double[] labels = getLabels(); + double pivot = features[(from + to) / 2][col]; int i = from, j = to; @@ -131,19 +129,11 @@ private void sort(int col, int from, int to) { } /** */ - public double[][] getFeatures() { - return features; - } - - /** */ - public double[] getLabels() { - return labels; - } - public double[] getCopyOfOriginalLabels() { return copyOfOriginalLabels; } + /** */ public void setCopyOfOriginalLabels(double[] copyOfOriginalLabels) { this.copyOfOriginalLabels = copyOfOriginalLabels; } @@ -170,7 +160,7 @@ public TreeDataIndex createIndexByFilter(int depth, TreeFilter filter) { if (depth == indexesCache.size()) { if (depth == 0) - indexesCache.add(new TreeDataIndex(features, labels)); + indexesCache.add(new TreeDataIndex(getFeatures(), getLabels())); else { TreeDataIndex lastIndex = indexesCache.get(depth - 1); indexesCache.add(lastIndex.filter(filter)); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java index 3e340f6047600..89b8c9c67d5eb 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java @@ -22,11 +22,13 @@ import java.util.function.BiFunction; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.convergence.mean.MeanAbsValueConvergenceCheckerFactory; +import org.apache.ignite.ml.composition.boosting.convergence.simple.ConvergenceCheckerStubFactory; import org.apache.ignite.ml.composition.predictionsaggregator.WeightedPredictionsAggregator; import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.primitives.vector.Vector; import org.apache.ignite.ml.math.primitives.vector.VectorUtils; -import org.apache.ignite.ml.trainers.DatasetTrainer; import org.apache.ignite.ml.tree.DecisionTreeConditionalNode; import org.apache.ignite.ml.tree.boosting.GDBBinaryClassifierOnTreesTrainer; import org.apache.ignite.ml.tree.boosting.GDBRegressionOnTreesTrainer; @@ -54,8 +56,8 @@ public void testFitRegression() { learningSample.put(i, new double[] {xs[i], ys[i]}); } - DatasetTrainer trainer - = new GDBRegressionOnTreesTrainer(1.0, 2000, 3, 0.0).withUseIndex(true); + GDBTrainer trainer = new GDBRegressionOnTreesTrainer(1.0, 2000, 3, 0.0) + .withUseIndex(true); Model mdl = trainer.fit( learningSample, 1, @@ -74,7 +76,6 @@ public void testFitRegression() { assertEquals(0.0, mse, 0.0001); - assertTrue(mdl instanceof ModelsComposition); ModelsComposition composition = (ModelsComposition)mdl; assertTrue(composition.toString().length() > 0); assertTrue(composition.toString(true).length() > 0); @@ -84,6 +85,13 @@ public void testFitRegression() { assertEquals(2000, composition.getModels().size()); assertTrue(composition.getPredictionsAggregator() instanceof WeightedPredictionsAggregator); + + trainer = trainer.withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.1)); + assertTrue(trainer.fit( + learningSample, 1, + (k, v) -> VectorUtils.of(v[0]), + (k, v) -> v[1] + ).getModels().size() < 2000); } /** */ @@ -107,7 +115,7 @@ public void testFitClassifierWithLearningStrategy() { } /** */ - private void testClassifier(BiFunction, + private void testClassifier(BiFunction, Model> fitter) { int sampleSize = 100; double[] xs = new double[sampleSize]; @@ -122,8 +130,9 @@ private void testClassifier(BiFunction mdl = fitter.apply(trainer, learningSample); @@ -132,7 +141,7 @@ private void testClassifier(BiFunction assertTrue(m instanceof DecisionTreeConditionalNode)); - assertEquals(500, composition.getModels().size()); + assertTrue(composition.getModels().size() < 500); assertTrue(composition.getPredictionsAggregator() instanceof WeightedPredictionsAggregator); + + trainer = trainer.withCheckConvergenceStgyFactory(new ConvergenceCheckerStubFactory()); + assertEquals(500, ((ModelsComposition)fitter.apply(trainer, learningSample)).getModels().size()); + } + + /** */ + @Test + public void testUpdate() { + int sampleSize = 100; + double[] xs = new double[sampleSize]; + double[] ys = new double[sampleSize]; + + for (int i = 0; i < sampleSize; i++) { + xs[i] = i; + ys[i] = ((int)(xs[i] / 10.0) % 2) == 0 ? -1.0 : 1.0; + } + + Map learningSample = new HashMap<>(); + for (int i = 0; i < sampleSize; i++) + learningSample.put(i, new double[] {xs[i], ys[i]}); + IgniteBiFunction fExtr = (k, v) -> VectorUtils.of(v[0]); + IgniteBiFunction lExtr = (k, v) -> v[1]; + + GDBTrainer classifTrainer = new GDBBinaryClassifierOnTreesTrainer(0.3, 500, 3, 0.0) + .withUseIndex(true) + .withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.3)); + GDBTrainer regressTrainer = new GDBRegressionOnTreesTrainer(0.3, 500, 3, 0.0) + .withUseIndex(true) + .withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.3)); + + testUpdate(learningSample, fExtr, lExtr, classifTrainer); + testUpdate(learningSample, fExtr, lExtr, regressTrainer); + } + + /** */ + private void testUpdate(Map dataset, IgniteBiFunction fExtr, + IgniteBiFunction lExtr, GDBTrainer trainer) { + + ModelsComposition originalMdl = trainer.fit(dataset, 1, fExtr, lExtr); + ModelsComposition updatedOnSameDataset = trainer.update(originalMdl, dataset, 1, fExtr, lExtr); + + LocalDatasetBuilder epmtyDataset = new LocalDatasetBuilder<>(new HashMap<>(), 1); + ModelsComposition updatedOnEmptyDataset = trainer.updateModel(originalMdl, epmtyDataset, fExtr, lExtr); + + dataset.forEach((k,v) -> { + Vector features = fExtr.apply(k, v); + + Double originalAnswer = originalMdl.apply(features); + Double updatedMdlAnswer1 = updatedOnSameDataset.apply(features); + Double updatedMdlAnswer2 = updatedOnEmptyDataset.apply(features); + + assertEquals(originalAnswer, updatedMdlAnswer1, 0.01); + assertEquals(originalAnswer, updatedMdlAnswer2, 0.01); + }); } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerTest.java new file mode 100644 index 0000000000000..50fdf8b14db8f --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/ConvergenceCheckerTest.java @@ -0,0 +1,82 @@ +/* + * 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.ml.composition.boosting.convergence; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.composition.ModelsComposition; +import org.apache.ignite.ml.composition.boosting.loss.Loss; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Before; + +/** */ +public abstract class ConvergenceCheckerTest { + /** Not converged model. */ + protected ModelsComposition notConvergedMdl = new ModelsComposition(Collections.emptyList(), null) { + @Override public Double apply(Vector features) { + return 2.1 * features.get(0); + } + }; + + /** Converged model. */ + protected ModelsComposition convergedMdl = new ModelsComposition(Collections.emptyList(), null) { + @Override public Double apply(Vector features) { + return 2 * (features.get(0) + 1); + } + }; + + /** Features extractor. */ + protected IgniteBiFunction fExtr = (x, y) -> VectorUtils.of(x); + + /** Label extractor. */ + protected IgniteBiFunction lbExtr = (x, y) -> y; + + /** Data. */ + protected Map data; + + /** */ + @Before + public void setUp() throws Exception { + data = new HashMap<>(); + for(int i = 0; i < 10; i ++) + data.put(new double[]{i, i + 1}, (double)(2 * (i + 1))); + } + + /** */ + public ConvergenceChecker createChecker(ConvergenceCheckerFactory factory, + LocalDatasetBuilder datasetBuilder) { + + return factory.create(data.size(), + x -> x, + new Loss() { + @Override public double error(long sampleSize, double lb, double mdlAnswer) { + return mdlAnswer - lb; + } + + @Override public double gradient(long sampleSize, double lb, double mdlAnswer) { + return mdlAnswer - lb; + } + }, + datasetBuilder, fExtr, lbExtr + ); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerTest.java new file mode 100644 index 0000000000000..0b42db814f779 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/mean/MeanAbsValueConvergenceCheckerTest.java @@ -0,0 +1,73 @@ +/* + * 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.ml.composition.boosting.convergence.mean; + +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceCheckerTest; +import org.apache.ignite.ml.dataset.impl.local.LocalDataset; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapData; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapDataBuilder; +import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Assert; +import org.junit.Test; + +/** */ +public class MeanAbsValueConvergenceCheckerTest extends ConvergenceCheckerTest { + /** */ + @Test + public void testConvergenceChecking() { + LocalDatasetBuilder datasetBuilder = new LocalDatasetBuilder<>(data, 1); + ConvergenceChecker checker = createChecker( + new MeanAbsValueConvergenceCheckerFactory(0.1), datasetBuilder); + + double error = checker.computeError(VectorUtils.of(1, 2), 4.0, notConvergedMdl); + Assert.assertEquals(1.9, error, 0.01); + Assert.assertFalse(checker.isConverged(datasetBuilder, notConvergedMdl)); + Assert.assertTrue(checker.isConverged(datasetBuilder, convergedMdl)); + + try(LocalDataset dataset = datasetBuilder.build( + new EmptyContextBuilder<>(), new FeatureMatrixWithLabelsOnHeapDataBuilder<>(fExtr, lbExtr))) { + + double onDSError = checker.computeMeanErrorOnDataset(dataset, notConvergedMdl); + Assert.assertEquals(1.55, onDSError, 0.01); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** Mean error more sensitive to anomalies in data */ + @Test + public void testConvergenceCheckingWithAnomaliesInData() { + data.put(new double[]{10, 11}, 100000.0); + LocalDatasetBuilder datasetBuilder = new LocalDatasetBuilder<>(data, 1); + ConvergenceChecker checker = createChecker( + new MeanAbsValueConvergenceCheckerFactory(0.1), datasetBuilder); + + try(LocalDataset dataset = datasetBuilder.build( + new EmptyContextBuilder<>(), new FeatureMatrixWithLabelsOnHeapDataBuilder<>(fExtr, lbExtr))) { + + double onDSError = checker.computeMeanErrorOnDataset(dataset, notConvergedMdl); + Assert.assertEquals(9090.41, onDSError, 0.01); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerTest.java new file mode 100644 index 0000000000000..d6880b48d4178 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/convergence/median/MedianOfMedianConvergenceCheckerTest.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.ml.composition.boosting.convergence.median; + +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceChecker; +import org.apache.ignite.ml.composition.boosting.convergence.ConvergenceCheckerTest; +import org.apache.ignite.ml.dataset.impl.local.LocalDataset; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapData; +import org.apache.ignite.ml.dataset.primitive.FeatureMatrixWithLabelsOnHeapDataBuilder; +import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Assert; +import org.junit.Test; + +/** */ +public class MedianOfMedianConvergenceCheckerTest extends ConvergenceCheckerTest { + /** */ + @Test + public void testConvergenceChecking() { + data.put(new double[]{10, 11}, 100000.0); + LocalDatasetBuilder datasetBuilder = new LocalDatasetBuilder<>(data, 1); + + ConvergenceChecker checker = createChecker( + new MedianOfMedianConvergenceCheckerFactory(0.1), datasetBuilder); + + double error = checker.computeError(VectorUtils.of(1, 2), 4.0, notConvergedMdl); + Assert.assertEquals(1.9, error, 0.01); + Assert.assertFalse(checker.isConverged(datasetBuilder, notConvergedMdl)); + Assert.assertTrue(checker.isConverged(datasetBuilder, convergedMdl)); + + try(LocalDataset dataset = datasetBuilder.build( + new EmptyContextBuilder<>(), new FeatureMatrixWithLabelsOnHeapDataBuilder<>(fExtr, lbExtr))) { + + double onDSError = checker.computeMeanErrorOnDataset(dataset, notConvergedMdl); + Assert.assertEquals(1.6, onDSError, 0.01); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java index f88fd3e6c2404..b06fd6745b6bf 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java @@ -17,185 +17,44 @@ package org.apache.ignite.ml.environment; -import java.util.Arrays; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; -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.QueryCursor; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.composition.ModelsComposition; import org.apache.ignite.ml.dataset.feature.FeatureMeta; import org.apache.ignite.ml.environment.logging.ConsoleLogger; import org.apache.ignite.ml.environment.logging.MLLogger; +import org.apache.ignite.ml.environment.parallelism.DefaultParallelismStrategy; import org.apache.ignite.ml.environment.parallelism.ParallelismStrategy; -import org.apache.ignite.ml.math.primitives.vector.VectorUtils; import org.apache.ignite.ml.tree.randomforest.RandomForestRegressionTrainer; import org.apache.ignite.ml.tree.randomforest.data.FeaturesCountSelectionStrategies; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.apache.ignite.thread.IgniteThread; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; /** * Tests for {@link LearningEnvironment} that require to start the whole Ignite infrastructure. IMPL NOTE based on * RandomForestRegressionExample example. */ -public class LearningEnvironmentTest extends GridCommonAbstractTest { - /** Number of nodes in grid */ - private static final int NODE_COUNT = 1; - - /** Ignite instance. */ - private Ignite ignite; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() { - stopAllGrids(); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() { - /* Grid instance. */ - ignite = grid(NODE_COUNT); - ignite.configuration().setPeerClassLoadingEnabled(true); - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - } - +public class LearningEnvironmentTest { /** */ + @Test public void testBasic() throws InterruptedException { - AtomicReference actualAmount = new AtomicReference<>(null); - AtomicReference actualMse = new AtomicReference<>(null); - AtomicReference actualMae = new AtomicReference<>(null); - - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - LearningEnvironmentTest.class.getSimpleName(), () -> { - IgniteCache dataCache = getTestCache(ignite); - - AtomicInteger idx = new AtomicInteger(0); - RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer( - IntStream.range(0, data[0].length - 1).mapToObj( - x -> new FeatureMeta("", idx.getAndIncrement(), false)).collect(Collectors.toList()) - ).withCountOfTrees(101) - .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) - .withMaxDepth(4) - .withMinImpurityDelta(0.) - .withSubsampleSize(0.3) - .withSeed(0); - - trainer.setEnvironment(LearningEnvironment.builder() - .withParallelismStrategy(ParallelismStrategy.Type.ON_DEFAULT_POOL) - .withLoggingFactory(ConsoleLogger.factory(MLLogger.VerboseLevel.LOW)) - .build() - ); - - ModelsComposition randomForest = trainer.fit(ignite, dataCache, - (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), - (k, v) -> v[v.length - 1] - ); - - double mse = 0.0; - double mae = 0.0; - int totalAmount = 0; - - try (QueryCursor> observations = dataCache.query(new ScanQuery<>())) { - for (Cache.Entry observation : observations) { - double difference = estimatePrediction(randomForest, observation); - - mse += Math.pow(difference, 2.0); - mae += Math.abs(difference); - - totalAmount++; - } - } - - actualAmount.set(totalAmount); - - mse = mse / totalAmount; - actualMse.set(mse); - - mae = mae / totalAmount; - actualMae.set(mae); - }); - - igniteThread.start(); - igniteThread.join(); - - assertEquals("Total amount", 23, (int)actualAmount.get()); - assertTrue("Mean squared error (MSE)", actualMse.get() > 0); - assertTrue("Mean absolute error (MAE)", actualMae.get() > 0); + RandomForestRegressionTrainer trainer = new RandomForestRegressionTrainer( + IntStream.range(0, 0).mapToObj( + x -> new FeatureMeta("", 0, false)).collect(Collectors.toList()) + ).withCountOfTrees(101) + .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) + .withMaxDepth(4) + .withMinImpurityDelta(0.) + .withSubsampleSize(0.3) + .withSeed(0); + + LearningEnvironment environment = LearningEnvironment.builder() + .withParallelismStrategy(ParallelismStrategy.Type.ON_DEFAULT_POOL) + .withLoggingFactory(ConsoleLogger.factory(MLLogger.VerboseLevel.LOW)) + .build(); + trainer.setEnvironment(environment); + assertEquals(DefaultParallelismStrategy.class, environment.parallelismStrategy().getClass()); + assertEquals(ConsoleLogger.class, environment.logger().getClass()); } - - /** */ - private double estimatePrediction(ModelsComposition randomForest, Cache.Entry observation) { - double[] val = observation.getValue(); - double[] inputs = Arrays.copyOfRange(val, 0, val.length - 1); - double groundTruth = val[val.length - 1]; - - double prediction = randomForest.apply(VectorUtils.of(inputs)); - - return prediction - groundTruth; - } - - /** - * Fills cache with data and returns it. - * - * @param ignite Ignite instance. - * @return Filled Ignite Cache. - */ - private IgniteCache getTestCache(Ignite ignite) { - CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); - cacheConfiguration.setName(UUID.randomUUID().toString()); - cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10)); - - IgniteCache cache = ignite.createCache(cacheConfiguration); - - for (int i = 0; i < data.length; i++) - cache.put(i, data[i]); - - return cache; - } - - /** - * Part of the Boston housing dataset. - */ - private static final double[][] data = { - {0.02731,0.00,7.070,0,0.4690,6.4210,78.90,4.9671,2,242.0,17.80,396.90,9.14,21.60}, - {0.02729,0.00,7.070,0,0.4690,7.1850,61.10,4.9671,2,242.0,17.80,392.83,4.03,34.70}, - {0.03237,0.00,2.180,0,0.4580,6.9980,45.80,6.0622,3,222.0,18.70,394.63,2.94,33.40}, - {0.06905,0.00,2.180,0,0.4580,7.1470,54.20,6.0622,3,222.0,18.70,396.90,5.33,36.20}, - {0.02985,0.00,2.180,0,0.4580,6.4300,58.70,6.0622,3,222.0,18.70,394.12,5.21,28.70}, - {0.08829,12.50,7.870,0,0.5240,6.0120,66.60,5.5605,5,311.0,15.20,395.60,12.43,22.90}, - {0.14455,12.50,7.870,0,0.5240,6.1720,96.10,5.9505,5,311.0,15.20,396.90,19.15,27.10}, - {0.21124,12.50,7.870,0,0.5240,5.6310,100.00,6.0821,5,311.0,15.20,386.63,29.93,16.50}, - {0.17004,12.50,7.870,0,0.5240,6.0040,85.90,6.5921,5,311.0,15.20,386.71,17.10,18.90}, - {0.22489,12.50,7.870,0,0.5240,6.3770,94.30,6.3467,5,311.0,15.20,392.52,20.45,15.00}, - {0.11747,12.50,7.870,0,0.5240,6.0090,82.90,6.2267,5,311.0,15.20,396.90,13.27,18.90}, - {0.09378,12.50,7.870,0,0.5240,5.8890,39.00,5.4509,5,311.0,15.20,390.50,15.71,21.70}, - {0.62976,0.00,8.140,0,0.5380,5.9490,61.80,4.7075,4,307.0,21.00,396.90,8.26,20.40}, - {0.63796,0.00,8.140,0,0.5380,6.0960,84.50,4.4619,4,307.0,21.00,380.02,10.26,18.20}, - {0.62739,0.00,8.140,0,0.5380,5.8340,56.50,4.4986,4,307.0,21.00,395.62,8.47,19.90}, - {1.05393,0.00,8.140,0,0.5380,5.9350,29.30,4.4986,4,307.0,21.00,386.85,6.58,23.10}, - {0.78420,0.00,8.140,0,0.5380,5.9900,81.70,4.2579,4,307.0,21.00,386.75,14.67,17.50}, - {0.80271,0.00,8.140,0,0.5380,5.4560,36.60,3.7965,4,307.0,21.00,288.99,11.69,20.20}, - {0.72580,0.00,8.140,0,0.5380,5.7270,69.50,3.7965,4,307.0,21.00,390.95,11.28,18.20}, - {1.25179,0.00,8.140,0,0.5380,5.5700,98.10,3.7979,4,307.0,21.00,376.57,21.02,13.60}, - {0.85204,0.00,8.140,0,0.5380,5.9650,89.20,4.0123,4,307.0,21.00,392.53,13.83,19.60}, - {1.23247,0.00,8.140,0,0.5380,6.1420,91.70,3.9769,4,307.0,21.00,396.90,18.72,15.20}, - {0.98843,0.00,8.140,0,0.5380,5.8130,100.00,4.0952,4,307.0,21.00,394.54,19.88,14.50} - }; - } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java index d8fb620d16663..199644bbb7a05 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/ANNClassificationTest.java @@ -93,17 +93,21 @@ public void testUpdate() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(NNStrategy.SIMPLE); - ANNClassificationModel updatedOnSameDataset = trainer.withSeed(1234L).update(originalMdl, + ANNClassificationModel updatedOnSameDataset = (ANNClassificationModel) trainer.withSeed(1234L).update(originalMdl, cacheMock, parts, (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), (k, v) -> v[2] - ); + ).withK(3) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(NNStrategy.SIMPLE); - ANNClassificationModel updatedOnEmptyDataset = trainer.withSeed(1234L).update(originalMdl, + ANNClassificationModel updatedOnEmptyDataset = (ANNClassificationModel) trainer.withSeed(1234L).update(originalMdl, new HashMap(), parts, (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), (k, v) -> v[2] - ); + ).withK(3) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(NNStrategy.SIMPLE); Vector v1 = VectorUtils.of(550, 550); Vector v2 = VectorUtils.of(-550, -550); From e134740c7c15df03ba9690b202ab2b5640a4863e Mon Sep 17 00:00:00 2001 From: Nikolai Kulagin <31161239+zzzadruga@users.noreply.github.com> Date: Thu, 6 Sep 2018 12:43:08 +0300 Subject: [PATCH 55/95] IGNITE-8936: Remove AffinityAssignment#clientEventChange as not used. - Fixes #4448. Signed-off-by: Nikolay Izhikov --- .../processors/affinity/AffinityAssignment.java | 6 ------ .../affinity/GridAffinityAssignment.java | 15 --------------- .../affinity/HistoryAffinityAssignment.java | 9 --------- 3 files changed, 30 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java index 1c321d3020b99..eb06d3e60b873 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java @@ -28,12 +28,6 @@ * Cached affinity calculations. */ public interface AffinityAssignment { - /** - * @return {@code True} if related discovery event did not not cause affinity assignment change and - * this assignment is just reference to the previous one. - */ - public boolean clientEventChange(); - /** * @return Affinity assignment computed by affinity function. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java index f96bc9d0c6fab..c4dfa8b683298 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java @@ -65,9 +65,6 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable /** */ private transient List> idealAssignment; - /** */ - private final boolean clientEvtChange; - /** * Constructs cached affinity calculations item. * @@ -78,7 +75,6 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable primary = new HashMap<>(); backup = new HashMap<>(); mvccCrd = null; - clientEvtChange = false; } /** @@ -101,7 +97,6 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable primary = new HashMap<>(); backup = new HashMap<>(); - clientEvtChange = false; initPrimaryBackupMaps(); } @@ -118,16 +113,6 @@ public class GridAffinityAssignment implements AffinityAssignment, Serializable primary = aff.primary; backup = aff.backup; mvccCrd = aff.mvccCrd; - - clientEvtChange = true; - } - - /** - * @return {@code True} if related discovery event did not not cause affinity assignment change and - * this assignment is just reference to the previous one. - */ - public boolean clientEventChange() { - return clientEvtChange; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java index 1a6b2efea9006..a9496486fb965 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java @@ -42,9 +42,6 @@ public class HistoryAffinityAssignment implements AffinityAssignment { /** */ private final List> idealAssignment; - /** */ - private final boolean clientEvtChange; - /** */ private final MvccCoordinator mvccCrd; @@ -56,7 +53,6 @@ public class HistoryAffinityAssignment implements AffinityAssignment { this.assignment = assign.assignment(); this.idealAssignment = assign.idealAssignment(); this.mvccCrd = assign.mvccCoordinator(); - this.clientEvtChange = assign.clientEventChange(); } /** {@inheritDoc} */ @@ -64,11 +60,6 @@ public class HistoryAffinityAssignment implements AffinityAssignment { return mvccCrd; } - /** {@inheritDoc} */ - @Override public boolean clientEventChange() { - return clientEvtChange; - } - /** {@inheritDoc} */ @Override public List> idealAssignment() { return idealAssignment; From 48287e9b3c6fe6287df0dd67f21a0fdfdc3a0cf1 Mon Sep 17 00:00:00 2001 From: Ivan Daschinskiy Date: Thu, 6 Sep 2018 12:35:05 +0300 Subject: [PATCH 56/95] IGNITE-9422 Fixed NPE on clients when new binary meta from joined node arrived. - Fixes #4681. Signed-off-by: Alexey Goncharuk --- .../CacheObjectBinaryProcessorImpl.java | 9 +- ...eObjectBinaryProcessorOnDiscoveryTest.java | 125 ++++++++++++++++++ .../ignite/testsuites/IgnitePdsTestSuite.java | 4 + 3 files changed, 135 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheObjectBinaryProcessorOnDiscoveryTest.java 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 d6b920a87e7fb..4c101b290fd54 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 @@ -492,7 +492,8 @@ public GridBinaryMarshaller marshaller() { try { BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); - metadataFileStore.mergeAndWriteMetadata(mergedMeta); + if (!ctx.clientNode()) + metadataFileStore.mergeAndWriteMetadata(mergedMeta); metadataLocCache.put(typeId, new BinaryMetadataHolder(mergedMeta, 0, 0)); } @@ -1016,7 +1017,8 @@ private IgniteNodeValidationResult validateBinaryMetadata(UUID rmtNodeId, Map("name", String.class)); + + stopGrid(0); + + Ignite ig1 = grid(1); + + // Modify existing type. + addBinaryType(ig1, "test_1", new IgniteBiTuple<>("id", Integer.class)); + + // Add new type. + addBinaryType(ig1, "test_2", new IgniteBiTuple<>("name", String.class)); + + stopGrid(1); + + startGrid(0); + + IgniteEx client = startGrid(getConfiguration("client")); + + startGrid(1); + + awaitPartitionMapExchange(); + + // Check that new metadata from grid_1 was handled without NPE on client. + assertNull(client.context().failure().failureContext()); + + // Check that metadata from grid_1 correctly loaded on client. + assertTrue(client.binary().type("test_1").fieldNames().containsAll(Arrays.asList("id", "name"))); + assertTrue(client.binary().type("test_2").fieldNames().contains("name")); + } + + /** + * @param ig Ig. + * @param typeName Type name. + * @param fields Fields. + */ + @SafeVarargs + private final BinaryObject addBinaryType(Ignite ig, String typeName, IgniteBiTuple>... fields) { + BinaryObjectBuilder builder = ig.binary().builder(typeName); + + if (fields != null) { + for (IgniteBiTuple> field: fields) + builder.setField(field.get1(), field.get2()); + } + + return builder.build(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java index 0a995da013a25..9f50b328916b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java @@ -20,6 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.pagemem.impl.PageMemoryNoLoadSelfTest; import org.apache.ignite.internal.processors.cache.IgniteClusterActivateDeactivateTestWithPersistence; +import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheObjectBinaryProcessorOnDiscoveryTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDestroyCacheTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDestroyCacheWithoutCheckpointsTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheConfigurationFileConsistencyCheckTest; @@ -95,6 +96,9 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(ExplicitWalDeltaConsistencyTest.class); suite.addTestSuite(SysPropWalDeltaConsistencyTest.class); + // Binary meta tests. + suite.addTestSuite(IgnitePdsCacheObjectBinaryProcessorOnDiscoveryTest.class); + return suite; } From a73c644225d06fa3cd10abb87d7a9b137634d9c6 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Thu, 6 Sep 2018 13:01:21 +0300 Subject: [PATCH 57/95] IGNITE-9093 Fixed IgniteDbPutGetWithCacheStoreTest failing due to node shutdown without checkpoint on stop - Fixes #4545. Signed-off-by: Alexey Goncharuk --- .../database/IgniteDbPutGetWithCacheStoreTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java index 3238acffa9687..8cca7f40e6ac1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetWithCacheStoreTest.java @@ -31,7 +31,6 @@ import org.apache.ignite.configuration.IgniteReflectionFactory; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; @@ -124,7 +123,7 @@ private void checkWriteThrough(CacheAtomicityMode atomicityMode) throws Exceptio assertEquals(2000, storeMap.size()); - stopAllGrids(); + stopAllGrids(false); storeMap.clear(); @@ -158,7 +157,7 @@ private void checkReadThrough(CacheAtomicityMode atomicityMode) throws Exception for (int i = 0; i < 2000; i++) assertEquals(i, ig.cache(CACHE_NAME).get(i)); - stopAllGrids(); + stopAllGrids(false); storeMap.clear(); From 295512c674854a08a832736cba4aeb1318761052 Mon Sep 17 00:00:00 2001 From: Nikolay Izhikov Date: Thu, 6 Sep 2018 13:05:39 +0300 Subject: [PATCH 58/95] IGNITE-8915: Check moved to GridQueryProcessor. Check for SqlFieldsQuery added. - Fixes #4414. Signed-off-by: Nikolay Izhikov --- .../cache/IgniteCacheProxyImpl.java | 3 - .../processors/query/GridQueryProcessor.java | 24 ++++++- .../IgniteCacheAbstractQuerySelfTest.java | 54 +++++++++++++++- .../IgniteCacheReplicatedQuerySelfTest.java | 27 -------- .../index/H2DynamicIndexAbstractSelfTest.java | 6 ++ .../local/IgniteCacheLocalQuerySelfTest.java | 64 ++++++++++++++++++- 6 files changed, 144 insertions(+), 34 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java index 69ea562f6433e..225fa817fa2e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java @@ -760,9 +760,6 @@ 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 (qry.isLocal() && (qry instanceof SqlQuery) && ctx.kernalContext().clientNode()) - throw new CacheException("Execution of local sql query on client node disallowed."); } /** {@inheritDoc} */ 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 bd5d91cc653b3..eb3f2a70c444b 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 @@ -2111,7 +2111,7 @@ public List>> querySqlFields(@Nullable final GridCache final boolean failOnMultipleStmts) { checkxEnabled(); - validateSqlFieldsQuery(qry); + validateSqlFieldsQuery(qry, ctx, cctx); if (!ctx.state().publicApiActiveState(true)) { throw new IgniteException("Can not perform the operation because the cluster is inactive. Note, that " + @@ -2161,13 +2161,31 @@ public List>> querySqlFields(@Nullable final GridCache * Validate SQL fields query. * * @param qry Query. + * @param ctx Kernal context. + * @param cctx Cache context. */ - private static void validateSqlFieldsQuery(SqlFieldsQuery qry) { + private static void validateSqlFieldsQuery(SqlFieldsQuery qry, GridKernalContext ctx, + @Nullable GridCacheContext cctx) { if (qry.isReplicatedOnly() && qry.getPartitions() != null) throw new CacheException("Partitions are not supported in replicated only mode."); if (qry.isDistributedJoins() && qry.getPartitions() != null) throw new CacheException("Using both partitions and distributed JOINs is not supported for the same query"); + + if (qry.isLocal() && ctx.clientNode() && (cctx == null || cctx.config().getCacheMode() != CacheMode.LOCAL)) + throw new CacheException("Execution of local SqlFieldsQuery on client node disallowed."); + } + + /** + * Validate SQL query. + * + * @param qry Query. + * @param ctx Kernal context. + * @param cctx Cache context. + */ + private static void validateSqlQuery(SqlQuery qry, GridKernalContext ctx, GridCacheContext cctx) { + if (qry.isLocal() && ctx.clientNode() && cctx.config().getCacheMode() != CacheMode.LOCAL) + throw new CacheException("Execution of local SqlQuery on client node disallowed."); } /** @@ -2238,6 +2256,8 @@ public List streamBatchedUpdateQuery(final String schemaName, final SqlCli */ public QueryCursor> querySql(final GridCacheContext cctx, final SqlQuery qry, boolean keepBinary) { + validateSqlQuery(qry, ctx, cctx); + if (qry.isReplicatedOnly() && qry.getPartitions() != null) throw new CacheException("Partitions are not supported in replicated only mode."); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index a845aaaef04d6..ac9de6f8101aa 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -64,6 +64,7 @@ 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.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.events.CacheQueryExecutedEvent; @@ -96,6 +97,7 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ; import static org.apache.ignite.internal.processors.cache.query.CacheQueryType.FULL_TEXT; import static org.apache.ignite.internal.processors.cache.query.CacheQueryType.SCAN; +import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause; import static org.junit.Assert.assertArrayEquals; /** @@ -142,9 +144,12 @@ protected NearCacheConfiguration nearCacheConfiguration() { c.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true).setIpFinder(ipFinder)); - if (igniteInstanceName.startsWith("client")) + if (igniteInstanceName.startsWith("client")) { c.setClientMode(true); + c.setDataStorageConfiguration(new DataStorageConfiguration()); + } + return c; } @@ -1773,6 +1778,53 @@ public void testFieldsQueryEvents() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testLocalSqlQueryFromClient() throws Exception { + try { + Ignite g = startGrid("client"); + + IgniteCache c = jcache(g, Integer.class, Integer.class); + + for (int i = 0; i < 10; i++) + c.put(i, i); + + SqlQuery qry = new SqlQuery<>(Integer.class, "_key >= 5 order by _key"); + + qry.setLocal(true); + + assertThrowsWithCause(() -> c.query(qry), CacheException.class); + } + finally { + stopGrid("client"); + } + } + + /** + * @throws Exception If failed. + */ + public void testLocalSqlFieldsQueryFromClient() throws Exception { + try { + Ignite g = startGrid("client"); + + IgniteCache c = jcache(g, UUID.class, Person.class); + + Person p = new Person("Jon", 1500); + + c.put(p.id(), p); + + SqlFieldsQuery qry = new SqlFieldsQuery("select count(*) from Person"); + + qry.setLocal(true); + + assertThrowsWithCause(() -> c.query(qry), CacheException.class); + } + finally { + stopGrid("client"); + } + } + /** * */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java index bd3dffdbded4b..13942c2ede3d4 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java @@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import javax.cache.Cache; -import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; @@ -53,7 +52,6 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.cache.CachePeekMode.ALL; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; -import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause; /** * Tests replicated query. @@ -162,31 +160,6 @@ public void testClientOnlyNode() throws Exception { } } - /** - * @throws Exception If failed. - */ - public void testClientsLocalQuery() throws Exception { - try { - Ignite g = startGrid("client"); - - IgniteCache c = jcache(g, Integer.class, Integer.class); - - for (int i = 0; i < 10; i++) - c.put(i, i); - - assertEquals(0, c.localSize()); - - SqlQuery qry = new SqlQuery<>(Integer.class, "_key >= 5 order by _key"); - - qry.setLocal(true); - - assertThrowsWithCause(() -> c.query(qry), CacheException.class); - } - finally { - stopGrid("client"); - } - } - /** * JUnit. * diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java index 9579b796d3f6d..10ef56fcd3b5e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicIndexAbstractSelfTest.java @@ -85,6 +85,9 @@ public void testCreateIndex() throws Exception { // Test that local queries on all nodes use new index. for (int i = 0 ; i < 4; i++) { + if (ignite(i).configuration().isClientMode()) + continue; + List> locRes = ignite(i).cache("cache").query(new SqlFieldsQuery("explain select \"id\" from " + "\"cache\".\"ValueClass\" where \"field1\" = 'A'").setLocal(true)).getAll(); @@ -155,6 +158,9 @@ public void testDropIndex() { // Test that no local queries on all nodes use new index. for (int i = 0 ; i < 4; i++) { + if (ignite(i).configuration().isClientMode()) + continue; + List> locRes = ignite(i).cache("cache").query(new SqlFieldsQuery("explain select \"id\" from " + "\"cache\".\"ValueClass\" where \"field1\" = 'A'").setLocal(true)).getAll(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java index 2570bc896c664..2272f27d158dd 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQuerySelfTest.java @@ -19,9 +19,12 @@ import java.util.Iterator; import java.util.List; +import java.util.UUID; import javax.cache.Cache; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; @@ -93,4 +96,63 @@ public void testQueryLocal() throws Exception { cache.destroy(); } } -} \ No newline at end of file + + /** {@inheritDoc} */ + @Override public void testLocalSqlQueryFromClient() throws Exception { + try { + Ignite g = startGrid("client"); + + IgniteCache c = jcache(g, Integer.class, Integer.class); + + for (int i = 0; i < 10; i++) + c.put(i, i); + + SqlQuery qry = new SqlQuery<>(Integer.class, "_key >= 5 order by _key"); + + qry.setLocal(true); + + try(QueryCursor> qryCursor = c.query(qry)) { + assertNotNull(qryCursor); + + List> res = qryCursor.getAll(); + + assertNotNull(res); + + assertEquals(5, res.size()); + } + } + finally { + stopGrid("client"); + } + } + + /** {@inheritDoc} */ + @Override public void testLocalSqlFieldsQueryFromClient() throws Exception { + try { + Ignite g = startGrid("client"); + + IgniteCache c = jcache(g, UUID.class, Person.class); + + Person p = new Person("Jon", 1500); + + c.put(p.id(), p); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from Person"); + + qry.setLocal(true); + + try(FieldsQueryCursor> qryCursor = c.query(qry)) { + assertNotNull(qryCursor); + + List> res = qryCursor.getAll(); + + assertNotNull(res); + + assertEquals(1, res.size()); + } + } + finally { + stopGrid("client"); + } + } +} From 5dc87e478fc90cade4c900a39ff1c47e26d73d68 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 6 Sep 2018 16:30:00 +0300 Subject: [PATCH 59/95] IGNITE-9479 Fixed spontaneous rebalance during cache start and improved logging level - Fixes #4691. Signed-off-by: Alexey Goncharuk --- .../dht/preloader/GridDhtForceKeysFuture.java | 36 ++-- .../preloader/GridDhtPartitionDemander.java | 22 +-- .../preloader/GridDhtPartitionSupplier.java | 4 +- .../GridDhtPartitionsExchangeFuture.java | 10 +- .../dht/preloader/GridDhtPreloader.java | 43 ++--- .../dht/IgniteCacheStartWithLoadTest.java | 165 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite7.java | 2 + 7 files changed, 213 insertions(+), 69 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheStartWithLoadTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java index 52691f02521b8..3b03958e7d099 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java @@ -274,8 +274,8 @@ private boolean map(Iterable keys, Collection exc) assert !n.id().equals(loc.id()); - if (log.isDebugEnabled()) - log.debug("Sending force key request [cacheName=" + cctx.name() + "node=" + n.id() + + if (log.isTraceEnabled()) + log.trace("Sending force key request [cacheName=" + cctx.name() + "node=" + n.id() + ", req=" + req + ']'); cctx.io().send(n, req, cctx.ioPolicy()); @@ -310,10 +310,10 @@ private Map> map(KeyCacheObject key, try { if (e != null && !e.isNewLocked()) { - if (log.isDebugEnabled()) { + if (log.isTraceEnabled()) { int part = cctx.affinity().partition(key); - log.debug("Will not rebalance key (entry is not new) [cacheName=" + cctx.name() + + log.trace("Will not rebalance key (entry is not new) [cacheName=" + cctx.name() + ", key=" + key + ", part=" + part + ", locId=" + cctx.nodeId() + ']'); } @@ -322,8 +322,8 @@ private Map> map(KeyCacheObject key, } } catch (GridCacheEntryRemovedException ignore) { - if (log.isDebugEnabled()) - log.debug("Received removed DHT entry for force keys request [entry=" + e + + if (log.isTraceEnabled()) + log.trace("Received removed DHT entry for force keys request [entry=" + e + ", locId=" + cctx.nodeId() + ']'); } @@ -333,8 +333,8 @@ private Map> map(KeyCacheObject key, new ArrayList<>(F.view(top.owners(part, topVer), F.notIn(exc))); if (owners.isEmpty() || (owners.contains(loc) && cctx.rebalanceEnabled())) { - if (log.isDebugEnabled()) - log.debug("Will not rebalance key (local node is owner) [key=" + key + ", part=" + part + + if (log.isTraceEnabled()) + log.trace("Will not rebalance key (local node is owner) [key=" + key + ", part=" + part + "topVer=" + topVer + ", locId=" + cctx.nodeId() + ']'); // Key is already rebalanced. @@ -344,8 +344,8 @@ private Map> map(KeyCacheObject key, // Create partition. GridDhtLocalPartition locPart = top.localPartition(part, topVer, false); - if (log.isDebugEnabled()) - log.debug("Mapping local partition [loc=" + cctx.localNodeId() + ", topVer" + topVer + + if (log.isTraceEnabled()) + log.trace("Mapping local partition [loc=" + cctx.localNodeId() + ", topVer" + topVer + ", part=" + locPart + ", owners=" + owners + ", allOwners=" + U.toShortString(top.owners(part)) + ']'); if (locPart == null) @@ -362,8 +362,8 @@ else if (!cctx.rebalanceEnabled() || locPart.state() == MOVING) { pick = F.first(F.view(owners, F.remoteNodes(loc.id()))); if (pick == null) { - if (log.isDebugEnabled()) - log.debug("Will not rebalance key (no nodes to request from with rebalancing disabled) [key=" + + if (log.isTraceEnabled()) + log.trace("Will not rebalance key (no nodes to request from with rebalancing disabled) [key=" + key + ", part=" + part + ", locId=" + cctx.nodeId() + ']'); return mappings; @@ -378,15 +378,15 @@ else if (!cctx.rebalanceEnabled() || locPart.state() == MOVING) { mappedKeys.add(key); - if (log.isDebugEnabled()) - log.debug("Will rebalance key from node [cacheName=" + cctx.name() + ", key=" + key + ", part=" + + if (log.isTraceEnabled()) + log.trace("Will rebalance key from node [cacheName=" + cctx.name() + ", key=" + key + ", part=" + part + ", node=" + pick.id() + ", locId=" + cctx.nodeId() + ']'); } else if (locPart.state() != OWNING) invalidParts.add(part); else { - if (log.isDebugEnabled()) - log.debug("Will not rebalance key (local partition is not MOVING) [cacheName=" + cctx.name() + + if (log.isTraceEnabled()) + log.trace("Will not rebalance key (local partition is not MOVING) [cacheName=" + cctx.name() + ", key=" + key + ", part=" + locPart + ", locId=" + cctx.nodeId() + ']'); } @@ -563,8 +563,8 @@ void onResult(GridDhtForceKeysResponse res) { return; } catch (GridCacheEntryRemovedException ignore) { - if (log.isDebugEnabled()) - log.debug("Trying to rebalance removed entry (will ignore) [cacheName=" + + if (log.isTraceEnabled()) + log.trace("Trying to rebalance removed entry (will ignore) [cacheName=" + cctx.name() + ", entry=" + entry + ']'); } finally { 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 a0f82eab217d0..851fcc9e2c26d 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 @@ -761,9 +761,9 @@ public void handleSupplyMessage( GridCacheEntryInfo entry = infos.next(); if (!preloadEntry(node, p, entry, topVer)) { - if (log.isDebugEnabled()) - log.debug("Got entries for invalid partition during " + - "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); + if (log.isTraceEnabled()) + log.trace("Got entries for invalid partition during " + + "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); break; } @@ -877,8 +877,8 @@ private boolean preloadEntry( cached = cctx.cache().entryEx(entry.key()); - if (log.isDebugEnabled()) - log.debug("Rebalancing key [key=" + entry.key() + ", part=" + p + ", node=" + from.id() + ']'); + if (log.isTraceEnabled()) + log.trace("Rebalancing key [key=" + entry.key() + ", part=" + p + ", node=" + from.id() + ']'); if (preloadPred == null || preloadPred.apply(entry)) { if (cached.initialValue( @@ -905,17 +905,17 @@ private boolean preloadEntry( else { cached.touch(topVer); // Start tracking. - if (log.isDebugEnabled()) - log.debug("Rebalancing entry is already in cache (will ignore) [key=" + cached.key() + + if (log.isTraceEnabled()) + log.trace("Rebalancing entry is already in cache (will ignore) [key=" + cached.key() + ", part=" + p + ']'); } } - else if (log.isDebugEnabled()) - log.debug("Rebalance predicate evaluated to false for entry (will ignore): " + entry); + else if (log.isTraceEnabled()) + log.trace("Rebalance predicate evaluated to false for entry (will ignore): " + entry); } catch (GridCacheEntryRemovedException ignored) { - if (log.isDebugEnabled()) - log.debug("Entry has been concurrently removed while rebalancing (will ignore) [key=" + + if (log.isTraceEnabled()) + log.trace("Entry has been concurrently removed while rebalancing (will ignore) [key=" + cached.key() + ", part=" + p + ']'); } catch (GridDhtInvalidPartitionException ignored) { 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 7ce4f7e7db496..524d02d32b78c 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 @@ -402,8 +402,8 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (preloadPred == null || preloadPred.apply(info)) s.addEntry0(part, iter.historical(part), info, grp.shared(), grp.cacheObjectContext()); else { - if (log.isDebugEnabled()) - log.debug("Rebalance predicate evaluated to false (will not send " + + if (log.isTraceEnabled()) + log.trace("Rebalance predicate evaluated to false (will not send " + "cache entry): " + info); } 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 357f3d78d0c06..e66511c1e1913 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 @@ -1225,8 +1225,8 @@ private void distributedExchange() throws IgniteCheckedException { boolean distributed = true; - // Do not perform distributed partition release in case of cluster activation or caches start. - if (activateCluster() || hasCachesToStart()) + // Do not perform distributed partition release in case of cluster activation. + if (activateCluster()) distributed = false; // On first phase we wait for finishing all local tx updates, atomic updates and lock releases on all nodes. @@ -2920,11 +2920,7 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe } } - // Don't validate partitions state in case of caches start. - boolean skipValidation = hasCachesToStart(); - - if (!skipValidation) - validatePartitionsState(); + validatePartitionsState(); if (firstDiscoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) { assert firstDiscoEvt instanceof DiscoveryCustomEvent; 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 7cf55a35536fc..f88676732d7b5 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 @@ -313,7 +313,7 @@ private boolean assignmentsChanged(AffinityTopologyVersion oldTopVer, AffinityTo msg.partitions().addHistorical(p, part.initialUpdateCounter(), countersMap.updateCounter(p), partCnt); } else { - Collection picked = pickOwners(p, topVer); + List picked = remoteOwners(p, topVer); if (picked.isEmpty()) { top.own(part); @@ -330,7 +330,7 @@ private boolean assignmentsChanged(AffinityTopologyVersion oldTopVer, AffinityTo log.debug("Owning partition as there are no other owners: " + part); } else { - ClusterNode n = F.rand(picked); + ClusterNode n = picked.get(0); GridDhtPartitionDemandMessage msg = assignments.get(n); @@ -359,42 +359,23 @@ private boolean assignmentsChanged(AffinityTopologyVersion oldTopVer, AffinityTo } /** - * Picks owners for specified partition {@code p} from affinity. + * Returns remote owners (excluding local node) for specified partition {@code p}. * * @param p Partition. * @param topVer Topology version. - * @return Picked owners. + * @return Nodes owning this partition. */ - private Collection pickOwners(int p, AffinityTopologyVersion topVer) { - Collection affNodes = grp.affinity().cachedAffinity(topVer).get(p); - - int affCnt = affNodes.size(); - - Collection rmts = remoteOwners(p, topVer); + private List remoteOwners(int p, AffinityTopologyVersion topVer) { + List owners = grp.topology().owners(p, topVer); - int rmtCnt = rmts.size(); + List res = new ArrayList<>(owners.size()); - if (rmtCnt <= affCnt) - return rmts; - - List sorted = new ArrayList<>(rmts); - - // Sort in descending order, so nodes with higher order will be first. - Collections.sort(sorted, CU.nodeComparator(false)); - - // Pick newest nodes. - return sorted.subList(0, affCnt); - } + for (ClusterNode owner : owners) { + if (!owner.id().equals(ctx.localNodeId())) + res.add(owner); + } - /** - * Returns remote owners (excluding local node) for specified partition {@code p}. - * - * @param p Partition. - * @param topVer Topology version. - * @return Nodes owning this partition. - */ - private Collection remoteOwners(int p, AffinityTopologyVersion topVer) { - return F.view(grp.topology().owners(p, topVer), F.remoteNodes(ctx.localNodeId())); + return res; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheStartWithLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheStartWithLoadTest.java new file mode 100644 index 0000000000000..acccc5be07fed --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheStartWithLoadTest.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht; + +import java.util.concurrent.ThreadLocalRandom; +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.cache.CacheAtomicityMode; +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.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.internal.U; +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.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.junit.Assert; + +/** + * + */ +public class IgniteCacheStartWithLoadTest extends GridCommonAbstractTest { + /** */ + static final String CACHE_NAME = "tx_repl"; + + @Override + protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setConsistentId(igniteInstanceName); + + CacheConfiguration ccfg = new CacheConfiguration().setName(CACHE_NAME) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setCacheMode(CacheMode.REPLICATED) + .setDataRegionName("ds") + .setAffinity(new RendezvousAffinityFunction(false, 32)); + + DataStorageConfiguration dsCfg = new DataStorageConfiguration() + .setDataRegionConfigurations( + new DataRegionConfiguration() + .setName("ds") + .setPersistenceEnabled(true) + .setMaxSize(1024 * 1024 * 1024) + ); + + cfg.setDataStorageConfiguration(dsCfg); + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** + * @throws Exception if failed. + */ + public void testNoRebalanceDuringCacheStart() throws Exception { + IgniteEx crd = (IgniteEx)startGrids(4); + + crd.cluster().active(true); + + AtomicBoolean txLoadStop = new AtomicBoolean(); + + AtomicInteger txLoaderNo = new AtomicInteger(0); + + IgniteInternalFuture txLoadFuture = GridTestUtils.runMultiThreadedAsync(() -> { + Ignite node = grid(txLoaderNo.getAndIncrement()); + IgniteCache cache = node.cache(CACHE_NAME); + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + final int keys = 5; + final int keysSpace = 10_000; + + while (!txLoadStop.get()) { + try (Transaction tx = node.transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) { + for (int it = 0; it < keys; it++) { + int key = rnd.nextInt(keysSpace); + byte[] value = new byte[2048]; + + cache.put(key, value); + } + tx.commit(); + + U.sleep(10); + } + catch (Throwable t) { + log.warning("Unexpected exception during tx load.", t); + } + } + }, 4, "tx-loader"); + + AtomicBoolean hasRebalance = new AtomicBoolean(); + + AtomicBoolean cacheRestartStop = new AtomicBoolean(); + + IgniteInternalFuture cacheRestartFuture = GridTestUtils.runAsync(() -> { + Ignite node = grid(0); + + final String tmpCacheName = "tmp"; + + while (!cacheRestartStop.get()) { + try { + node.getOrCreateCache(tmpCacheName); + + boolean hasMoving = false; + + for (int i = 0; i < 4; i++) { + hasMoving |= grid(i).cachex(CACHE_NAME).context().topology().hasMovingPartitions(); + } + + if (hasMoving) { + log.error("Cache restarter has been stopped because rebalance is triggered for stable caches."); + + hasRebalance.set(true); + + return; + } + + node.destroyCache(tmpCacheName); + + U.sleep(10_000); + } + catch (Throwable t) { + log.warning("Unexpected exception during caches restart.", t); + } + } + }); + + U.sleep(60_000); + + cacheRestartStop.set(true); + txLoadStop.set(true); + + cacheRestartFuture.get(); + txLoadFuture.get(); + + Assert.assertFalse(hasRebalance.get()); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 5 * 60 * 1000; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java index 3a3595098feeb..0a2f86e45b705 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.cache.distributed.Cache64kPartitionsTest; import org.apache.ignite.internal.processors.cache.distributed.CacheRentingStateRepairTest; import org.apache.ignite.internal.processors.cache.distributed.CacheDataLossOnPartitionMoveTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheStartWithLoadTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingPartitionCountersTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingWithAsyncClearingTest; import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionMultinodeMixedRegionsTest; @@ -65,6 +66,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { TestSuite suite = new TestSuite("IgniteCache With Persistence Test Suite"); suite.addTestSuite(CheckpointBufferDeadlockTest.class); + suite.addTestSuite(IgniteCacheStartWithLoadTest.class); suite.addTestSuite(AuthenticationConfigurationClusterTest.class); suite.addTestSuite(AuthenticationProcessorSelfTest.class); From e63243c288ac163a6481332fcea5008d974e3779 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 6 Sep 2018 18:03:50 +0300 Subject: [PATCH 60/95] IGNITE-9084 Fixed error handling for historical rebalance - Fixes #4437. Signed-off-by: Alexey Goncharuk --- .../communication/GridIoMessageFactory.java | 7 + .../processors/cache/GridCacheIoManager.java | 3 +- .../preloader/GridDhtPartitionDemander.java | 14 +- .../preloader/GridDhtPartitionSupplier.java | 69 +++++-- .../GridDhtPartitionSupplyMessage.java | 6 +- .../GridDhtPartitionSupplyMessageV2.java | 153 +++++++++++++++ .../persistence/GridCacheOffheapManager.java | 10 + .../db/wal/IgniteWalRebalanceTest.java | 185 +++++++++++++++++- 8 files changed, 427 insertions(+), 20 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index 8dddd8ba2ece5..2970e71c41339 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 @@ -54,6 +54,8 @@ import org.apache.ignite.internal.processors.cache.WalStateAckMessage; import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage; import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse; @@ -889,6 +891,11 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; + case 120: + msg = new GridDhtPartitionSupplyMessageV2(); + + break; + case 124: msg = new GridMessageCollection<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index 01344211f36e9..1e25c935f8104 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -914,7 +914,8 @@ private void processFailedMessage(UUID nodeId, break; - case 114: { + case 114: + case 120: { processMessage(nodeId, msg, c);// Will be handled by Rebalance Demander. } 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 851fcc9e2c26d..3c1090f9e713a 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 @@ -688,10 +688,20 @@ public void handleSupplyMessage( if (log.isDebugEnabled()) log.debug("Received supply message [grp=" + grp.cacheOrGroupName() + ", msg=" + supply + ']'); - // Check whether there were class loading errors on unmarshal + // Check whether there were error during supply message unmarshalling process. if (supply.classError() != null) { U.warn(log, "Rebalancing from node cancelled [grp=" + grp.cacheOrGroupName() + ", node=" + nodeId + - "]. Class got undeployed during preloading: " + supply.classError()); + "]. Supply message couldn't be unmarshalled: " + supply.classError()); + + fut.cancel(nodeId); + + return; + } + + // Check whether there were error during supplying process. + if (supply.error() != null) { + U.warn(log, "Rebalancing from node cancelled [grp=" + grp.cacheOrGroupName() + ", node=" + nodeId + + "]. Supplier has failed with error: " + supply.error()); fut.cancel(nodeId); 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 524d02d32b78c..2090c078f828c 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 @@ -218,9 +218,11 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (node == null) return; - try { - SupplyContext sctx; + IgniteRebalanceIterator iter = null; + + SupplyContext sctx = null; + try { synchronized (scMap) { sctx = scMap.remove(contextId); @@ -229,7 +231,7 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand scMap.put(contextId, sctx); if (log.isDebugEnabled()) - log.debug("Stale demand message [grp=" + grp.cacheOrGroupName() + log.debug("Stale demand message [cache=" + grp.cacheOrGroupName() + ", actualContext=" + sctx + ", from=" + nodeId + ", demandMsg=" + d + "]"); @@ -241,7 +243,7 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand // Demand request should not contain empty partitions if no supply context is associated with it. if (sctx == null && (d.partitions() == null || d.partitions().isEmpty())) { if (log.isDebugEnabled()) - log.debug("Empty demand message [grp=" + grp.cacheOrGroupName() + log.debug("Empty demand message [cache=" + grp.cacheOrGroupName() + ", from=" + nodeId + ", topicId=" + topicId + ", demandMsg=" + d + "]"); @@ -272,8 +274,6 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand d.topologyVersion(), grp.deploymentEnabled()); - IgniteRebalanceIterator iter; - Set remainingParts; if (sctx == null || sctx.iterator == null) { @@ -452,13 +452,56 @@ else if (iter.isPartitionMissing(p)) { ", topology=" + demTop + ", rebalanceId=" + d.rebalanceId() + ", topicId=" + topicId + "]"); } - catch (IgniteCheckedException e) { - U.error(log, "Failed to send partition supply message to node: " + nodeId, e); - } - catch (IgniteSpiException e) { - if (log.isDebugEnabled()) - log.debug("Failed to send message to node (current node is stopping?) [node=" + node.id() + - ", msg=" + e.getMessage() + ']'); + catch (Throwable t) { + if (grp.shared().kernalContext().isStopping()) + return; + + // Sending supply messages with error requires new protocol. + boolean sendErrMsg = node.version().compareTo(GridDhtPartitionSupplyMessageV2.AVAILABLE_SINCE) >= 0; + + if (t instanceof IgniteSpiException) { + if (log.isDebugEnabled()) + log.debug("Failed to send message to node (current node is stopping?) [node=" + node.id() + + ", msg=" + t.getMessage() + ']'); + + sendErrMsg = false; + } + else + U.error(log, "Failed to continue supplying process for " + + "[cache=" + grp.cacheOrGroupName() + ", node=" + nodeId + + ", topicId=" + contextId.get2() + ", topVer=" + contextId.get3() + "]", t); + + try { + if (sctx != null) + clearContext(sctx, log); + else if (iter != null) + iter.close(); + } + catch (Throwable t1) { + U.error(log, "Failed to cleanup supplying context " + + "[cache=" + grp.cacheOrGroupName() + ", node=" + nodeId + + ", topicId=" + contextId.get2() + ", topVer=" + contextId.get3() + "]", t1); + } + + if (!sendErrMsg) + return; + + try { + GridDhtPartitionSupplyMessageV2 errMsg = new GridDhtPartitionSupplyMessageV2( + d.rebalanceId(), + grp.groupId(), + d.topologyVersion(), + grp.deploymentEnabled(), + t + ); + + reply(node, d, errMsg, contextId); + } + catch (Throwable t1) { + U.error(log, "Failed to send supply error message for " + + "[cache=" + grp.cacheOrGroupName() + ", node=" + nodeId + + ", topicId=" + contextId.get2() + ", topVer=" + contextId.get3() + "]", t1); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java index 4ecffc492c7b4..284700ad3a44f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java @@ -89,10 +89,12 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple * @param topVer Topology version. * @param addDepInfo Deployment info flag. */ - GridDhtPartitionSupplyMessage(long rebalanceId, + GridDhtPartitionSupplyMessage( + long rebalanceId, int grpId, AffinityTopologyVersion topVer, - boolean addDepInfo) { + boolean addDepInfo + ) { this.grpId = grpId; this.rebalanceId = rebalanceId; this.topVer = topVer; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java new file mode 100644 index 0000000000000..a77576683815f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java @@ -0,0 +1,153 @@ +/* + * 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.preloader; + +import java.nio.ByteBuffer; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.Nullable; + +/** + * Supply message with supplier error transfer support. + */ +public class GridDhtPartitionSupplyMessageV2 extends GridDhtPartitionSupplyMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** Available since. */ + public static final IgniteProductVersion AVAILABLE_SINCE = IgniteProductVersion.fromString("2.7.0"); + + /** Supplying process error. */ + @GridDirectTransient + private Throwable err; + + /** Supplying process error bytes. */ + private byte[] errBytes; + + /** + * Default constructor. + */ + public GridDhtPartitionSupplyMessageV2() { + } + + /** + * @param rebalanceId Rebalance id. + * @param grpId Group id. + * @param topVer Topology version. + * @param addDepInfo Add dep info. + * @param err Supply process error. + */ + public GridDhtPartitionSupplyMessageV2( + long rebalanceId, + int grpId, + AffinityTopologyVersion topVer, + boolean addDepInfo, + Throwable err + ) { + super(rebalanceId, grpId, topVer, addDepInfo); + + this.err = err; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + if (err != null && errBytes == null) + errBytes = U.marshal(ctx, err); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + if (errBytes != null && err == null) + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 12: + if (!writer.writeByteArray("errBytes", errBytes)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 12: + errBytes = reader.readByteArray("errBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridDhtPartitionSupplyMessageV2.class); + } + + /** {@inheritDoc} */ + @Nullable @Override public Throwable error() { + return err; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 120; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 13; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index 4c45352ce5585..199efcb59446d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -880,6 +880,10 @@ private static class WALHistoricalIterator implements IgniteHistoricalIterator { /** Flag indicates that partition belongs to current {@link #next} is finished and no longer needs to rebalance. */ private boolean reachedPartitionEnd; + /** Flag indicates that update counters for requested partitions have been reached and done. + * It means that no further iteration is needed. */ + private boolean doneAllPartitions; + /** * @param grp Cache context. * @param walIt WAL iterator. @@ -953,6 +957,9 @@ private WALHistoricalIterator(CacheGroupContext grp, CachePartitionPartialCounte doneParts.add(next.partitionId()); reachedPartitionEnd = false; + + if (doneParts.size() == partMap.size()) + doneAllPartitions = true; } advance(); @@ -1011,6 +1018,9 @@ private void releasePartitions() { private void advance() { next = null; + if (doneAllPartitions) + return; + while (true) { if (entryIt != null) { while (entryIt.hasNext()) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java index d4f6f0cee4e82..57565bfd58c48 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java @@ -17,6 +17,10 @@ package org.apache.ignite.internal.processors.cache.persistence.db.wal; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.OpenOption; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -38,18 +42,31 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCachePreloader; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiPredicate; 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.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.READ; +import static java.nio.file.StandardOpenOption.WRITE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; /** @@ -62,6 +79,9 @@ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { /** Partitions count. */ private static final int PARTS_CNT = 32; + /** Block message predicate to set to Communication SPI in node configuration. */ + private IgniteBiPredicate blockMessagePredicate; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { System.setProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, "0"); //to make all rebalance wal-based @@ -92,6 +112,12 @@ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { cfg.setCommunicationSpi(new WalRebalanceCheckingCommunicationSpi()); + if (blockMessagePredicate != null) { + TestRecordingCommunicationSpi spi = (TestRecordingCommunicationSpi) cfg.getCommunicationSpi(); + + spi.blockMessages(blockMessagePredicate); + } + return cfg; } @@ -227,6 +253,8 @@ public void testWithLocalWalChange() throws Exception { cache.put(k, new IndexedObject(k - 1)); } + forceCheckpoint(); + stopAllGrids(); IgniteEx ig0 = (IgniteEx) startGrids(2); @@ -240,6 +268,8 @@ public void testWithLocalWalChange() throws Exception { for (int k = 0; k < entryCnt; k++) cache.put(k, new IndexedObject(k)); + forceCheckpoint(); + // This node should rebalance data from other nodes and shouldn't have WAL history. Ignite ignite = startGrid(2); @@ -258,6 +288,8 @@ else if (k % 3 == 1) // Spread removes across all partitions. cache.remove(k); } + forceCheckpoint(); + // Stop grids which have actual WAL history. stopGrid(0); @@ -309,6 +341,8 @@ public void testWithGlobalWalChange() throws Exception { cache.put(k, new IndexedObject(k - 1)); } + forceCheckpoint(); + stopAllGrids(); // Rewrite data with globally disabled WAL. @@ -325,6 +359,8 @@ public void testWithGlobalWalChange() throws Exception { for (int k = 0; k < entryCnt; k++) cache.put(k, new IndexedObject(k)); + forceCheckpoint(); + crd.cluster().enableWal(CACHE_NAME); // This node shouldn't rebalance data using WAL, because it was disabled on other nodes. @@ -364,6 +400,100 @@ public void testWithGlobalWalChange() throws Exception { } } + /** + * Tests that cache rebalance is cancelled if supplyer node got exception during iteration over WAL. + * + * @throws Exception If failed. + */ + public void testRebalanceCancelOnSupplyError() throws Exception { + // Prepare some data. + IgniteEx crd = (IgniteEx) startGrids(3); + + crd.cluster().active(true); + + final int entryCnt = PARTS_CNT * 10; + + { + IgniteCache cache = crd.cache(CACHE_NAME); + + for (int k = 0; k < entryCnt; k++) + cache.put(k, new IndexedObject(k - 1)); + } + + forceCheckpoint(); + + stopAllGrids(); + + // Rewrite data to trigger further rebalance. + IgniteEx supplierNode = (IgniteEx) startGrid(0); + + supplierNode.cluster().active(true); + + IgniteCache cache = supplierNode.cache(CACHE_NAME); + + for (int k = 0; k < entryCnt; k++) + cache.put(k, new IndexedObject(k)); + + forceCheckpoint(); + + final int groupId = supplierNode.cachex(CACHE_NAME).context().groupId(); + + // Delay rebalance process for specified group. + blockMessagePredicate = (node, msg) -> { + if (msg instanceof GridDhtPartitionDemandMessage) + return ((GridDhtPartitionDemandMessage) msg).groupId() == groupId; + + return false; + }; + + IgniteEx demanderNode = startGrid(2); + + AffinityTopologyVersion curTopVer = demanderNode.context().discovery().topologyVersionEx(); + + // Wait for rebalance process start on demander node. + final GridCachePreloader preloader = demanderNode.cachex(CACHE_NAME).context().group().preloader(); + + GridTestUtils.waitForCondition(() -> + ((GridDhtPartitionDemander.RebalanceFuture) preloader.rebalanceFuture()).topologyVersion().equals(curTopVer), + getTestTimeout() + ); + + // Inject I/O factory which can throw exception during WAL read on supplier node. + FailingIOFactory ioFactory = new FailingIOFactory(new RandomAccessFileIOFactory()); + + ((FileWriteAheadLogManager) supplierNode.cachex(CACHE_NAME).context().shared().wal()).setFileIOFactory(ioFactory); + + ioFactory.throwExceptionOnWalRead(); + + // Resume rebalance process. + TestRecordingCommunicationSpi spi = (TestRecordingCommunicationSpi) demanderNode.configuration().getCommunicationSpi(); + + spi.stopBlock(); + + // Wait till rebalance will be failed and cancelled. + Boolean result = preloader.rebalanceFuture().get(); + + Assert.assertEquals("Rebalance should be cancelled on demander node: " + preloader.rebalanceFuture(), false, result); + + // Stop blocking messages and fail WAL during read. + blockMessagePredicate = null; + + ioFactory.reset(); + + // Start last grid and wait for rebalance. + startGrid(1); + + awaitPartitionMapExchange(); + + // Check data consistency. + for (Ignite ig : G.allGrids()) { + IgniteCache cache1 = ig.cache(CACHE_NAME); + + for (int k = 0; k < entryCnt; k++) + assertEquals(new IndexedObject(k), cache1.get(k)); + } + } + /** * */ @@ -409,7 +539,7 @@ private IndexedObject(int iVal) { /** * Wrapper of communication spi to detect on what topology versions WAL rebalance has happened. */ - public static class WalRebalanceCheckingCommunicationSpi extends TcpCommunicationSpi { + public static class WalRebalanceCheckingCommunicationSpi extends TestRecordingCommunicationSpi { /** (Group ID, Set of topology versions). */ private static final Map> topVers = new HashMap<>(); @@ -464,4 +594,55 @@ public static void cleanup() { super.sendMessage(node, msg, ackC); } } + + /** + * + */ + static class FailingIOFactory implements FileIOFactory { + /** Fail read operations. */ + private volatile boolean failRead; + + /** Delegate. */ + private final FileIOFactory delegate; + + /** + * @param delegate Delegate. + */ + FailingIOFactory(FileIOFactory delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + return create(file, CREATE, WRITE, READ); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO delegateIO = delegate.create(file, modes); + + if (file.getName().endsWith(".wal") && failRead) + return new FileIODecorator(delegateIO) { + @Override public int read(ByteBuffer destBuf) throws IOException { + throw new IgniteException("Test exception."); + } + }; + + return delegateIO; + } + + /** + * + */ + public void throwExceptionOnWalRead() { + failRead = true; + } + + /** + * + */ + public void reset() { + failRead = false; + } + } } \ No newline at end of file From 4ca677ea25ad67a2b1f076b46a7297ed0ff91e38 Mon Sep 17 00:00:00 2001 From: Igor Seliverstov Date: Fri, 7 Sep 2018 11:57:46 +0300 Subject: [PATCH 61/95] IGNITE-9483: MVCC compatibility fixes. This closes #4696. --- .../ignite/codegen/MessageCodeGenerator.java | 4 +- .../internal/jdbc/thin/JdbcThinTcpIo.java | 25 ++++++----- .../managers/communication/GridIoManager.java | 2 +- .../dht/GridDhtTxFinishRequest.java | 24 +++++----- .../dht/GridDhtTxPrepareRequest.java | 44 +++++++++---------- .../GridDhtPartitionsSingleMessage.java | 40 ++++++++--------- .../distributed/near/GridNearGetRequest.java | 24 +++++----- .../near/GridNearSingleGetRequest.java | 16 +++---- .../near/GridNearTxPrepareResponse.java | 24 +++++----- .../cache/query/GridCacheQueryRequest.java | 44 +++++++++---------- .../odbc/jdbc/JdbcBatchExecuteRequest.java | 40 ++++++++--------- .../odbc/jdbc/JdbcBatchExecuteResult.java | 11 +++-- .../odbc/jdbc/JdbcBulkLoadAckResult.java | 11 +++-- .../odbc/jdbc/JdbcBulkLoadBatchRequest.java | 11 +++-- .../processors/odbc/jdbc/JdbcColumnMeta.java | 7 ++- .../odbc/jdbc/JdbcColumnMetaV2.java | 11 +++-- .../odbc/jdbc/JdbcColumnMetaV3.java | 11 +++-- .../odbc/jdbc/JdbcColumnMetaV4.java | 11 +++-- .../odbc/jdbc/JdbcConnectionContext.java | 6 ++- .../processors/odbc/jdbc/JdbcIndexMeta.java | 7 ++- .../odbc/jdbc/JdbcMessageParser.java | 13 ++++-- .../odbc/jdbc/JdbcMetaColumnsRequest.java | 11 +++-- .../odbc/jdbc/JdbcMetaColumnsResult.java | 15 ++++--- .../odbc/jdbc/JdbcMetaIndexesRequest.java | 11 +++-- .../odbc/jdbc/JdbcMetaIndexesResult.java | 15 ++++--- .../odbc/jdbc/JdbcMetaParamsRequest.java | 11 +++-- .../odbc/jdbc/JdbcMetaParamsResult.java | 15 ++++--- .../odbc/jdbc/JdbcMetaPrimaryKeysRequest.java | 11 +++-- .../odbc/jdbc/JdbcMetaPrimaryKeysResult.java | 15 ++++--- .../odbc/jdbc/JdbcMetaSchemasRequest.java | 11 +++-- .../odbc/jdbc/JdbcMetaSchemasResult.java | 11 +++-- .../odbc/jdbc/JdbcMetaTablesRequest.java | 11 +++-- .../odbc/jdbc/JdbcMetaTablesResult.java | 15 ++++--- .../jdbc/JdbcOrderedBatchExecuteRequest.java | 9 ++-- .../jdbc/JdbcOrderedBatchExecuteResult.java | 11 +++-- .../odbc/jdbc/JdbcParameterMeta.java | 7 ++- .../odbc/jdbc/JdbcPrimaryKeyMeta.java | 7 ++- .../processors/odbc/jdbc/JdbcQuery.java | 7 ++- .../odbc/jdbc/JdbcQueryCloseRequest.java | 11 +++-- ...cQueryExecuteMultipleStatementsResult.java | 15 ++++--- .../odbc/jdbc/JdbcQueryExecuteRequest.java | 33 +++++++------- .../odbc/jdbc/JdbcQueryExecuteResult.java | 11 +++-- .../odbc/jdbc/JdbcQueryFetchRequest.java | 11 +++-- .../odbc/jdbc/JdbcQueryFetchResult.java | 11 +++-- .../odbc/jdbc/JdbcQueryMetadataRequest.java | 11 +++-- .../odbc/jdbc/JdbcQueryMetadataResult.java | 15 ++++--- .../odbc/jdbc/JdbcRawBinarylizable.java | 8 +++- .../processors/odbc/jdbc/JdbcRequest.java | 13 ++++-- .../processors/odbc/jdbc/JdbcResponse.java | 11 +++-- .../processors/odbc/jdbc/JdbcResult.java | 12 +++-- .../processors/odbc/jdbc/JdbcResultInfo.java | 7 ++- .../processors/odbc/jdbc/JdbcTableMeta.java | 7 ++- .../odbc/odbc/OdbcConnectionContext.java | 2 +- .../odbc/odbc/OdbcMessageParser.java | 4 +- .../h2/twostep/msg/GridH2QueryRequest.java | 44 +++++++++---------- 55 files changed, 466 insertions(+), 339 deletions(-) diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java index 51365f39766ae..9512bae49654c 100644 --- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java +++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java @@ -38,8 +38,6 @@ import java.util.Set; import java.util.TreeSet; import java.util.UUID; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryEnlistRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistRequest; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.GridCodegenConverter; import org.apache.ignite.internal.GridDirectCollection; @@ -170,7 +168,7 @@ public static void main(String[] args) throws Exception { // gen.generateAll(true); - gen.generateAndWrite(GridNearTxQueryResultsEnlistRequest.class); +// gen.generateAndWrite(GridNearTxQueryResultsEnlistRequest.class); // gen.generateAndWrite(GridNearAtomicUpdateRequest.class); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java index 2c3f321a9585e..61554232adf39 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java @@ -74,7 +74,7 @@ public class JdbcThinTcpIo { private static final ClientListenerProtocolVersion VER_2_7_0 = ClientListenerProtocolVersion.create(2, 7, 0); /** Current version. */ - private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_7_0; + public static final ClientListenerProtocolVersion CURRENT_VER = VER_2_7_0; /** Initial output stream capacity for handshake. */ private static final int HANDSHAKE_MSG_SIZE = 13; @@ -326,7 +326,9 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL writer.writeBoolean(connProps.isAutoCloseServerCursor()); writer.writeBoolean(connProps.isLazy()); writer.writeBoolean(connProps.isSkipReducerOnUpdate()); - writer.writeString(connProps.nestedTxMode()); + + if (ver.compareTo(VER_2_7_0) >= 0) + writer.writeString(connProps.nestedTxMode()); if (!F.isEmpty(connProps.getUsername())) { assert ver.compareTo(VER_2_5_0) >= 0 : "Authentication is supported since 2.5"; @@ -375,15 +377,16 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL + ", url=" + connProps.getUrl() + ']', SqlStateCode.CONNECTION_REJECTED); } - if (VER_2_4_0.equals(srvProtocolVer) - || VER_2_3_0.equals(srvProtocolVer) - || VER_2_1_5.equals(srvProtocolVer)) - handshake(srvProtocolVer); - else if (VER_2_1_0.equals(srvProtocolVer)) + if (VER_2_5_0.equals(srvProtoVer0) + || VER_2_4_0.equals(srvProtoVer0) + || VER_2_3_0.equals(srvProtoVer0) + || VER_2_1_5.equals(srvProtoVer0)) + handshake(srvProtoVer0); + else if (VER_2_1_0.equals(srvProtoVer0)) handshake_2_1_0(); else { throw new SQLException("Handshake failed [driverProtocolVer=" + CURRENT_VER + - ", remoteNodeProtocolVer=" + srvProtocolVer + ", err=" + err + ']', + ", remoteNodeProtocolVer=" + srvProtoVer0 + ", err=" + err + ']', SqlStateCode.CONNECTION_REJECTED); } } @@ -466,7 +469,7 @@ void sendBatchRequestNoWaitResponse(JdbcOrderedBatchExecuteRequest req) throws I BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(cap), null, null); - req.writeBinary(writer); + req.writeBinary(writer, srvProtocolVer); send(writer.array()); } @@ -500,7 +503,7 @@ JdbcResponse sendRequest(JdbcRequest req) throws SQLException, IOException { BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(cap), null, null); - req.writeBinary(writer); + req.writeBinary(writer, srvProtocolVer); send(writer.array()); @@ -523,7 +526,7 @@ JdbcResponse readResponse() throws IOException { JdbcResponse res = new JdbcResponse(); - res.readBinary(reader); + res.readBinary(reader, srvProtocolVer); return res; } 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 f515d579ceadf..b3c80b05f635b 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 @@ -1662,7 +1662,7 @@ else if (async) throw (ClusterTopologyCheckedException)e.getCause(); if (!ctx.discovery().alive(node)) - throw new ClusterTopologyCheckedException("Failed to send message, node left: " + node.id()); + throw new ClusterTopologyCheckedException("Failed to send message, node left: " + node.id(), e); throw new IgniteCheckedException("Failed to send message (node may have left the grid or " + "TCP connection cannot be established due to firewall issues) " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java index 2696c0e0621d4..b1908ff4120ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java @@ -397,37 +397,37 @@ public Map updateCountersMap() { writer.incrementState(); case 23: - if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + if (!writer.writeUuid("nearNodeId", nearNodeId)) return false; writer.incrementState(); case 24: - if (!writer.writeUuid("nearNodeId", nearNodeId)) + if (!writer.writeMessage("partUpdateCnt", partUpdateCnt)) return false; writer.incrementState(); case 25: - if (!writer.writeMessage("partUpdateCnt", partUpdateCnt)) + if (!writer.writeCollection("pendingVers", pendingVers, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 26: - if (!writer.writeCollection("pendingVers", pendingVers, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("writeVer", writeVer)) return false; writer.incrementState(); case 27: - if (!writer.writeMap("updCntrs", updCntrs, MessageCollectionItemType.INT, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); case 28: - if (!writer.writeMessage("writeVer", writeVer)) + if (!writer.writeMap("updCntrs", updCntrs, MessageCollectionItemType.INT, MessageCollectionItemType.MSG)) return false; writer.incrementState(); @@ -469,7 +469,7 @@ public Map updateCountersMap() { reader.incrementState(); case 23: - mvccSnapshot = reader.readMessage("mvccSnapshot"); + nearNodeId = reader.readUuid("nearNodeId"); if (!reader.isLastRead()) return false; @@ -477,7 +477,7 @@ public Map updateCountersMap() { reader.incrementState(); case 24: - nearNodeId = reader.readUuid("nearNodeId"); + partUpdateCnt = reader.readMessage("partUpdateCnt"); if (!reader.isLastRead()) return false; @@ -485,7 +485,7 @@ public Map updateCountersMap() { reader.incrementState(); case 25: - partUpdateCnt = reader.readMessage("partUpdateCnt"); + pendingVers = reader.readCollection("pendingVers", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -493,7 +493,7 @@ public Map updateCountersMap() { reader.incrementState(); case 26: - pendingVers = reader.readCollection("pendingVers", MessageCollectionItemType.MSG); + writeVer = reader.readMessage("writeVer"); if (!reader.isLastRead()) return false; @@ -501,7 +501,7 @@ public Map updateCountersMap() { reader.incrementState(); case 27: - updCntrs = reader.readMap("updCntrs", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -509,7 +509,7 @@ public Map updateCountersMap() { reader.incrementState(); case 28: - writeVer = reader.readMessage("writeVer"); + updCntrs = reader.readMap("updCntrs", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false); if (!reader.isLastRead()) return false; 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 a71fb6c96e653..68c1f39564e83 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 @@ -427,67 +427,67 @@ public boolean skipCompletedVersion() { writer.incrementState(); case 23: - if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + if (!writer.writeUuid("nearNodeId", nearNodeId)) return false; writer.incrementState(); case 24: - if (!writer.writeUuid("nearNodeId", nearNodeId)) + if (!writer.writeCollection("nearWrites", nearWrites, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 25: - if (!writer.writeCollection("nearWrites", nearWrites, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("nearXidVer", nearXidVer)) return false; writer.incrementState(); case 26: - if (!writer.writeMessage("nearXidVer", nearXidVer)) + if (!writer.writeCollection("ownedKeys", ownedKeys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 27: - if (!writer.writeCollection("ownedKeys", ownedKeys, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("ownedVals", ownedVals, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 28: - if (!writer.writeCollection("ownedVals", ownedVals, MessageCollectionItemType.MSG)) + if (!writer.writeBitSet("preloadKeys", preloadKeys)) return false; writer.incrementState(); case 29: - if (!writer.writeBitSet("preloadKeys", preloadKeys)) + if (!writer.writeBoolean("skipCompletedVers", skipCompletedVers)) return false; writer.incrementState(); case 30: - if (!writer.writeBoolean("skipCompletedVers", skipCompletedVers)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 31: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; writer.incrementState(); case 32: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); case 33: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); @@ -533,7 +533,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 23: - mvccSnapshot = reader.readMessage("mvccSnapshot"); + nearNodeId = reader.readUuid("nearNodeId"); if (!reader.isLastRead()) return false; @@ -541,7 +541,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 24: - nearNodeId = reader.readUuid("nearNodeId"); + nearWrites = reader.readCollection("nearWrites", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -549,7 +549,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 25: - nearWrites = reader.readCollection("nearWrites", MessageCollectionItemType.MSG); + nearXidVer = reader.readMessage("nearXidVer"); if (!reader.isLastRead()) return false; @@ -557,7 +557,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 26: - nearXidVer = reader.readMessage("nearXidVer"); + ownedKeys = reader.readCollection("ownedKeys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -565,7 +565,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 27: - ownedKeys = reader.readCollection("ownedKeys", MessageCollectionItemType.MSG); + ownedVals = reader.readCollection("ownedVals", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -573,7 +573,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 28: - ownedVals = reader.readCollection("ownedVals", MessageCollectionItemType.MSG); + preloadKeys = reader.readBitSet("preloadKeys"); if (!reader.isLastRead()) return false; @@ -581,7 +581,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 29: - preloadKeys = reader.readBitSet("preloadKeys"); + skipCompletedVers = reader.readBoolean("skipCompletedVers"); if (!reader.isLastRead()) return false; @@ -589,7 +589,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 30: - skipCompletedVers = reader.readBoolean("skipCompletedVers"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -597,7 +597,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 31: - subjId = reader.readUuid("subjId"); + taskNameHash = reader.readInt("taskNameHash"); if (!reader.isLastRead()) return false; @@ -605,7 +605,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 32: - taskNameHash = reader.readInt("taskNameHash"); + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -613,7 +613,7 @@ public boolean skipCompletedVersion() { reader.incrementState(); case 33: - topVer = reader.readMessage("topVer"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; 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 77d7eabfc780e..7998e07c2bb35 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 @@ -474,61 +474,61 @@ public void setError(Exception ex) { switch (writer.state()) { case 5: - if (!writer.writeMessage("activeQryTrackers", activeQryTrackers)) + if (!writer.writeBoolean("client", client)) return false; writer.incrementState(); case 6: - if (!writer.writeBoolean("client", client)) + if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT)) return false; writer.incrementState(); case 7: - if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; writer.incrementState(); case 8: - if (!writer.writeByteArray("errBytes", errBytes)) + if (!writer.writeMessage("finishMsg", finishMsg)) return false; writer.incrementState(); case 9: - if (!writer.writeMessage("finishMsg", finishMsg)) + if (!writer.writeCollection("grpsAffRequest", grpsAffRequest, MessageCollectionItemType.INT)) return false; writer.incrementState(); case 10: - if (!writer.writeCollection("grpsAffRequest", grpsAffRequest, MessageCollectionItemType.INT)) + if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) return false; writer.incrementState(); case 11: - if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) + if (!writer.writeByteArray("partHistCntrsBytes", partHistCntrsBytes)) return false; writer.incrementState(); case 12: - if (!writer.writeByteArray("partHistCntrsBytes", partHistCntrsBytes)) + if (!writer.writeByteArray("partsBytes", partsBytes)) return false; writer.incrementState(); case 13: - if (!writer.writeByteArray("partsBytes", partsBytes)) + if (!writer.writeByteArray("partsSizesBytes", partsSizesBytes)) return false; writer.incrementState(); case 14: - if (!writer.writeByteArray("partsSizesBytes", partsSizesBytes)) + if (!writer.writeMessage("activeQryTrackers", activeQryTrackers)) return false; writer.incrementState(); @@ -549,7 +549,7 @@ public void setError(Exception ex) { switch (reader.state()) { case 5: - activeQryTrackers = reader.readMessage("activeQryTrackers"); + client = reader.readBoolean("client"); if (!reader.isLastRead()) return false; @@ -557,7 +557,7 @@ public void setError(Exception ex) { reader.incrementState(); case 6: - client = reader.readBoolean("client"); + dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false); if (!reader.isLastRead()) return false; @@ -565,7 +565,7 @@ public void setError(Exception ex) { reader.incrementState(); case 7: - dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false); + errBytes = reader.readByteArray("errBytes"); if (!reader.isLastRead()) return false; @@ -573,7 +573,7 @@ public void setError(Exception ex) { reader.incrementState(); case 8: - errBytes = reader.readByteArray("errBytes"); + finishMsg = reader.readMessage("finishMsg"); if (!reader.isLastRead()) return false; @@ -581,7 +581,7 @@ public void setError(Exception ex) { reader.incrementState(); case 9: - finishMsg = reader.readMessage("finishMsg"); + grpsAffRequest = reader.readCollection("grpsAffRequest", MessageCollectionItemType.INT); if (!reader.isLastRead()) return false; @@ -589,7 +589,7 @@ public void setError(Exception ex) { reader.incrementState(); case 10: - grpsAffRequest = reader.readCollection("grpsAffRequest", MessageCollectionItemType.INT); + partCntrsBytes = reader.readByteArray("partCntrsBytes"); if (!reader.isLastRead()) return false; @@ -597,7 +597,7 @@ public void setError(Exception ex) { reader.incrementState(); case 11: - partCntrsBytes = reader.readByteArray("partCntrsBytes"); + partHistCntrsBytes = reader.readByteArray("partHistCntrsBytes"); if (!reader.isLastRead()) return false; @@ -605,7 +605,7 @@ public void setError(Exception ex) { reader.incrementState(); case 12: - partHistCntrsBytes = reader.readByteArray("partHistCntrsBytes"); + partsBytes = reader.readByteArray("partsBytes"); if (!reader.isLastRead()) return false; @@ -613,7 +613,7 @@ public void setError(Exception ex) { reader.incrementState(); case 13: - partsBytes = reader.readByteArray("partsBytes"); + partsSizesBytes = reader.readByteArray("partsSizesBytes"); if (!reader.isLastRead()) return false; @@ -621,7 +621,7 @@ public void setError(Exception ex) { reader.incrementState(); case 14: - partsSizesBytes = reader.readByteArray("partsSizesBytes"); + activeQryTrackers = reader.readMessage("activeQryTrackers"); if (!reader.isLastRead()) return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java index c891bfb7dc3c8..f594e2bd353f6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java @@ -397,37 +397,37 @@ public long accessTtl() { writer.incrementState(); case 9: - if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + if (!writer.writeCollection("readersFlags", readersFlags, MessageCollectionItemType.BOOLEAN)) return false; writer.incrementState(); case 10: - if (!writer.writeCollection("readersFlags", readersFlags, MessageCollectionItemType.BOOLEAN)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 11: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; writer.incrementState(); case 12: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); case 13: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeMessage("ver", ver)) return false; writer.incrementState(); case 14: - if (!writer.writeMessage("ver", ver)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); @@ -497,7 +497,7 @@ public long accessTtl() { reader.incrementState(); case 9: - mvccSnapshot = reader.readMessage("mvccSnapshot"); + readersFlags = reader.readCollection("readersFlags", MessageCollectionItemType.BOOLEAN); if (!reader.isLastRead()) return false; @@ -505,7 +505,7 @@ public long accessTtl() { reader.incrementState(); case 10: - readersFlags = reader.readCollection("readersFlags", MessageCollectionItemType.BOOLEAN); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -513,7 +513,7 @@ public long accessTtl() { reader.incrementState(); case 11: - subjId = reader.readUuid("subjId"); + taskNameHash = reader.readInt("taskNameHash"); if (!reader.isLastRead()) return false; @@ -521,7 +521,7 @@ public long accessTtl() { reader.incrementState(); case 12: - taskNameHash = reader.readInt("taskNameHash"); + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -529,7 +529,7 @@ public long accessTtl() { reader.incrementState(); case 13: - topVer = reader.readMessage("topVer"); + ver = reader.readMessage("ver"); if (!reader.isLastRead()) return false; @@ -537,7 +537,7 @@ public long accessTtl() { reader.incrementState(); case 14: - ver = reader.readMessage("ver"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java index d1aa2e96b9496..cf885e27111e3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java @@ -337,7 +337,7 @@ public boolean recovery() { reader.incrementState(); case 8: - mvccSnapshot = reader.readMessage("mvccSnapshot"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -345,7 +345,7 @@ public boolean recovery() { reader.incrementState(); case 9: - subjId = reader.readUuid("subjId"); + taskNameHash = reader.readInt("taskNameHash"); if (!reader.isLastRead()) return false; @@ -353,7 +353,7 @@ public boolean recovery() { reader.incrementState(); case 10: - taskNameHash = reader.readInt("taskNameHash"); + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -361,7 +361,7 @@ public boolean recovery() { reader.incrementState(); case 11: - topVer = reader.readMessage("topVer"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; @@ -419,25 +419,25 @@ public boolean recovery() { writer.incrementState(); case 8: - if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 9: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; writer.incrementState(); case 10: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); case 11: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java index 92db4633b732a..e9865df5e3e3b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java @@ -407,37 +407,37 @@ public boolean hasOwnedValue(IgniteTxKey key) { writer.incrementState(); case 15: - if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 16: - if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 17: - if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 18: - if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("retVal", retVal)) return false; writer.incrementState(); case 19: - if (!writer.writeMessage("retVal", retVal)) + if (!writer.writeMessage("writeVer", writeVer)) return false; writer.incrementState(); case 20: - if (!writer.writeMessage("writeVer", writeVer)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); @@ -499,7 +499,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 15: - mvccSnapshot = reader.readMessage("mvccSnapshot"); + ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -507,7 +507,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 16: - ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG); + ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -515,7 +515,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 17: - ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG); + pending = reader.readCollection("pending", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -523,7 +523,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 18: - pending = reader.readCollection("pending", MessageCollectionItemType.MSG); + retVal = reader.readMessage("retVal"); if (!reader.isLastRead()) return false; @@ -531,7 +531,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 19: - retVal = reader.readMessage("retVal"); + writeVer = reader.readMessage("writeVer"); if (!reader.isLastRead()) return false; @@ -539,7 +539,7 @@ public boolean hasOwnedValue(IgniteTxKey key) { reader.incrementState(); case 20: - writeVer = reader.readMessage("writeVer"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; 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 ebbca351aae06..b7205b6e17bbb 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 @@ -586,55 +586,55 @@ public int taskHash() { writer.incrementState(); case 15: - if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + if (!writer.writeInt("pageSize", pageSize)) return false; writer.incrementState(); case 16: - if (!writer.writeInt("pageSize", pageSize)) + if (!writer.writeInt("part", part)) return false; writer.incrementState(); case 17: - if (!writer.writeInt("part", part)) + if (!writer.writeByteArray("rdcBytes", rdcBytes)) return false; writer.incrementState(); case 18: - if (!writer.writeByteArray("rdcBytes", rdcBytes)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 19: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeInt("taskHash", taskHash)) return false; writer.incrementState(); case 20: - if (!writer.writeInt("taskHash", taskHash)) + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); case 21: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeByteArray("transBytes", transBytes)) return false; writer.incrementState(); case 22: - if (!writer.writeByteArray("transBytes", transBytes)) + if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1)) return false; writer.incrementState(); case 23: - if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); @@ -752,7 +752,7 @@ public int taskHash() { reader.incrementState(); case 15: - mvccSnapshot = reader.readMessage("mvccSnapshot"); + pageSize = reader.readInt("pageSize"); if (!reader.isLastRead()) return false; @@ -760,7 +760,7 @@ public int taskHash() { reader.incrementState(); case 16: - pageSize = reader.readInt("pageSize"); + part = reader.readInt("part"); if (!reader.isLastRead()) return false; @@ -768,7 +768,7 @@ public int taskHash() { reader.incrementState(); case 17: - part = reader.readInt("part"); + rdcBytes = reader.readByteArray("rdcBytes"); if (!reader.isLastRead()) return false; @@ -776,7 +776,7 @@ public int taskHash() { reader.incrementState(); case 18: - rdcBytes = reader.readByteArray("rdcBytes"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -784,7 +784,7 @@ public int taskHash() { reader.incrementState(); case 19: - subjId = reader.readUuid("subjId"); + taskHash = reader.readInt("taskHash"); if (!reader.isLastRead()) return false; @@ -792,7 +792,7 @@ public int taskHash() { reader.incrementState(); case 20: - taskHash = reader.readInt("taskHash"); + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -800,7 +800,7 @@ public int taskHash() { reader.incrementState(); case 21: - topVer = reader.readMessage("topVer"); + transBytes = reader.readByteArray("transBytes"); if (!reader.isLastRead()) return false; @@ -808,23 +808,23 @@ public int taskHash() { reader.incrementState(); case 22: - transBytes = reader.readByteArray("transBytes"); + byte typeOrd; + + typeOrd = reader.readByte("type"); if (!reader.isLastRead()) return false; + type = GridCacheQueryType.fromOrdinal(typeOrd); + reader.incrementState(); case 23: - byte typeOrd; - - typeOrd = reader.readByte("type"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; - type = GridCacheQueryType.fromOrdinal(typeOrd); - reader.incrementState(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java index 065a29880c8ad..404a1c931b477 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java @@ -17,17 +17,20 @@ package org.apache.ignite.internal.processors.odbc.jdbc; -import java.io.IOException; import java.util.ArrayList; import java.util.List; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; 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.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_4_0; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_7_0; + /** * JDBC batch execute request. */ @@ -129,8 +132,8 @@ public boolean isLastStreamBatch() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); @@ -138,20 +141,23 @@ public boolean isLastStreamBatch() { writer.writeInt(queries.size()); for (JdbcQuery q : queries) - q.writeBinary(writer); + q.writeBinary(writer, ver); } else writer.writeInt(0); - writer.writeBoolean(autoCommit); - writer.writeBoolean(lastStreamBatch); + if (ver.compareTo(VER_2_4_0) >= 0) + writer.writeBoolean(lastStreamBatch); + + if (ver.compareTo(VER_2_7_0) >= 0) + writer.writeBoolean(autoCommit); } /** {@inheritDoc} */ @SuppressWarnings("SimplifiableIfStatement") - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); schemaName = reader.readString(); @@ -162,22 +168,16 @@ public boolean isLastStreamBatch() { for (int i = 0; i < n; ++i) { JdbcQuery qry = new JdbcQuery(); - qry.readBinary(reader); + qry.readBinary(reader, ver); queries.add(qry); } - try { - if (reader.available() > 0) { - autoCommit = reader.readBoolean(); - lastStreamBatch = reader.readBoolean(); - } - else - autoCommit = true; - } - catch (IOException e) { - throw new BinaryObjectException(e); - } + if (ver.compareTo(VER_2_4_0) >= 0) + lastStreamBatch = reader.readBoolean(); + + if (ver.compareTo(VER_2_7_0) >= 0) + autoCommit = reader.readBoolean(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java index 3fc9dd7c5ab21..0d93244e487ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -97,8 +98,9 @@ public String errorMessage() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeInt(errCode); writer.writeString(errMsg); @@ -107,8 +109,9 @@ public String errorMessage() { /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); errCode = reader.readInt(); errMsg = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java index e670baffcf371..b0750fd5f6aab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadAckResult.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.util.typedef.internal.S; @@ -81,8 +82,9 @@ public BulkLoadAckClientParameters params() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(qryId); writer.writeString(params.localFileName()); @@ -90,8 +92,9 @@ public BulkLoadAckClientParameters params() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); qryId = reader.readLong(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java index 7db4951eee508..347a5df0c98c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBulkLoadBatchRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.NotNull; @@ -143,8 +144,9 @@ public int cmd() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(qryId); writer.writeInt(batchIdx); @@ -153,8 +155,9 @@ public int cmd() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); qryId = reader.readLong(); batchIdx = reader.readInt(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java index 5b6304d627582..2b08fb4809588 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl; import org.apache.ignite.internal.jdbc.thin.JdbcThinUtils; import org.apache.ignite.internal.jdbc2.JdbcUtils; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -158,7 +159,8 @@ public boolean isNullable() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) { writer.writeString(schemaName); writer.writeString(tblName); writer.writeString(colName); @@ -169,7 +171,8 @@ public boolean isNullable() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) { schemaName = reader.readString(); tblName = reader.readString(); colName = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java index a2b4acf0f40cb..6c77b55aa3047 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java @@ -19,6 +19,7 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -54,15 +55,17 @@ public JdbcColumnMetaV2(String schemaName, String tblName, String colName, Class } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) { + super.writeBinary(writer, ver); writer.writeBoolean(nullable); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) { + super.readBinary(reader, ver); nullable = reader.readBoolean(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV3.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV3.java index 9911be0a3e1c8..8f8adfe683c11 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV3.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV3.java @@ -19,6 +19,7 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -63,15 +64,17 @@ public JdbcColumnMetaV3(String schemaName, String tblName, String colName, Class } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) { + super.writeBinary(writer, ver); writer.writeString(dfltValue); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) { + super.readBinary(reader, ver); dfltValue = reader.readString(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java index ec76983cd4507..6bdc597bd65fe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java @@ -19,6 +19,7 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -68,16 +69,18 @@ public JdbcColumnMetaV4(String schemaName, String tblName, String colName, Class } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) { + super.writeBinary(writer, ver); writer.writeInt(precision); writer.writeInt(scale); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) { + super.readBinary(reader, ver); precision = reader.readInt(); scale = reader.readInt(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java index 9f3fe2a6277bc..5e9a1b3d33715 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java @@ -142,7 +142,7 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin if (ver.compareTo(VER_2_3_0) >= 0) skipReducerOnUpdate = reader.readBoolean(); - if (ver.compareTo(VER_2_5_0) >= 0) { + if (ver.compareTo(VER_2_7_0) >= 0) { String nestedTxModeName = reader.readString(); if (!F.isEmpty(nestedTxModeName)) { @@ -153,7 +153,9 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin throw new IgniteCheckedException("Invalid nested transactions handling mode: " + nestedTxModeName); } } + } + if (ver.compareTo(VER_2_5_0) >= 0) { String user = null; String passwd = null; @@ -170,7 +172,7 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin actx = authenticate(user, passwd); } - parser = new JdbcMessageParser(ctx); + parser = new JdbcMessageParser(ctx, ver); JdbcResponseSender sender = new JdbcResponseSender() { @Override public void send(ClientListenerResponse resp) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java index d33f8871b7292..d7de6d7355bd4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java @@ -24,6 +24,7 @@ import org.apache.ignite.cache.QueryIndexType; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -123,7 +124,8 @@ public List fieldsAsc() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { writer.writeString(schemaName); writer.writeString(tblName); writer.writeString(idxName); @@ -142,7 +144,8 @@ public List fieldsAsc() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { schemaName = reader.readString(); tblName = reader.readString(); idxName = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java index 7a9c2ab0d91de..1718c00bf4c57 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; import org.apache.ignite.internal.binary.streams.BinaryInputStream; import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.ClientListenerRequest; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; @@ -34,14 +35,20 @@ public class JdbcMessageParser implements ClientListenerMessageParser { /** Kernal context. */ private final GridKernalContext ctx; + /** Client protocol version. */ + private final ClientListenerProtocolVersion ver; + /** Initial output stream capacity. */ protected static final int INIT_CAP = 1024; /** * @param ctx Context. + * @param ver Client protocol version. */ - public JdbcMessageParser(GridKernalContext ctx) { + public JdbcMessageParser(GridKernalContext ctx, + ClientListenerProtocolVersion ver) { this.ctx = ctx; + this.ver = ver; } /** @@ -68,7 +75,7 @@ protected BinaryWriterExImpl createWriter(int cap) { BinaryReaderExImpl reader = createReader(msg); - return JdbcRequest.readRequest(reader); + return JdbcRequest.readRequest(reader, ver); } /** {@inheritDoc} */ @@ -81,7 +88,7 @@ protected BinaryWriterExImpl createWriter(int cap) { BinaryWriterExImpl writer = createWriter(INIT_CAP); - res.writeBinary(writer); + res.writeBinary(writer, ver); return writer.array(); }} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java index fca1bf70d2b8e..389629e95c817 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -78,8 +79,9 @@ public String columnName() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); writer.writeString(tblName); @@ -87,8 +89,9 @@ public String columnName() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); schemaName = reader.readString(); tblName = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java index 9931ce0f6a38a..199c76d42dfcc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java @@ -24,6 +24,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -77,8 +78,9 @@ public List meta() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); if (F.isEmpty(meta)) writer.writeInt(0); @@ -86,13 +88,14 @@ public List meta() { writer.writeInt(meta.size()); for(JdbcColumnMeta m : meta) - m.writeBinary(writer); + m.writeBinary(writer, ver); } } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); int size = reader.readInt(); @@ -104,7 +107,7 @@ public List meta() { for (int i = 0; i < size; ++i) { JdbcColumnMeta m = createMetaColumn(); - m.readBinary(reader); + m.readBinary(reader, ver); meta.add(m); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java index d4a53d8e79399..28fe558df3180 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -66,16 +67,18 @@ public String tableName() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); writer.writeString(tblName); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); schemaName = reader.readString(); tblName = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java index 2316dfceac416..b18095409a9c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java @@ -24,6 +24,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -57,8 +58,9 @@ public List meta() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); if (F.isEmpty(meta)) writer.writeInt(0); @@ -66,13 +68,14 @@ public List meta() { writer.writeInt(meta.size()); for(JdbcIndexMeta m : meta) - m.writeBinary(writer); + m.writeBinary(writer, ver); } } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); int size = reader.readInt(); @@ -84,7 +87,7 @@ public List meta() { for (int i = 0; i < size; ++i) { JdbcIndexMeta m = new JdbcIndexMeta(); - m.readBinary(reader); + m.readBinary(reader, ver); meta.add(m); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java index 6b955f9cd4845..360f17db41eb8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -65,16 +66,18 @@ public String sql() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); writer.writeString(sql); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); schemaName = reader.readString(); sql = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java index 7563e014934cd..43c242289d7dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java @@ -23,6 +23,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -49,8 +50,9 @@ public class JdbcMetaParamsResult extends JdbcResult { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); if (F.isEmpty(meta)) writer.writeInt(0); @@ -58,13 +60,14 @@ public class JdbcMetaParamsResult extends JdbcResult { writer.writeInt(meta.size()); for(JdbcParameterMeta m : meta) - m.writeBinary(writer); + m.writeBinary(writer, ver); } } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); int size = reader.readInt(); @@ -76,7 +79,7 @@ public class JdbcMetaParamsResult extends JdbcResult { for (int i = 0; i < size; ++i) { JdbcParameterMeta m = new JdbcParameterMeta(); - m.readBinary(reader); + m.readBinary(reader, ver); meta.add(m); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java index 957225a380ef7..def53b84ab065 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -66,16 +67,18 @@ public String tableName() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); writer.writeString(tblName); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); schemaName = reader.readString(); tblName = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java index bd0dd90857240..f6d986eadbf36 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java @@ -24,6 +24,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -51,8 +52,9 @@ public class JdbcMetaPrimaryKeysResult extends JdbcResult { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); if (F.isEmpty(meta)) writer.writeInt(0); @@ -60,13 +62,14 @@ public class JdbcMetaPrimaryKeysResult extends JdbcResult { writer.writeInt(meta.size()); for(JdbcPrimaryKeyMeta m : meta) - m.writeBinary(writer); + m.writeBinary(writer, ver); } } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); int size = reader.readInt(); @@ -78,7 +81,7 @@ public class JdbcMetaPrimaryKeysResult extends JdbcResult { for (int i = 0; i < size; ++i) { JdbcPrimaryKeyMeta m = new JdbcPrimaryKeyMeta(); - m.readBinary(reader); + m.readBinary(reader, ver); meta.add(m); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java index 43bbe5dbbea11..715c798059a73 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -53,15 +54,17 @@ public String schemaName() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); this.schemaName = reader.readString(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java index 48b6aae387171..8d24a4121bc2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -46,15 +47,17 @@ public class JdbcMetaSchemasResult extends JdbcResult { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); JdbcUtils.writeStringCollection(writer, schemas); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); schemas = JdbcUtils.readStringList(reader); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java index 740b6561600c5..3ea6c35853063 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -65,16 +66,18 @@ public String tableName() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); writer.writeString(tblName); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); this.schemaName = reader.readString(); this.tblName = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java index 585667ef4cf2f..d5afb7ca2031a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java @@ -23,6 +23,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -49,8 +50,9 @@ public class JdbcMetaTablesResult extends JdbcResult { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); if (F.isEmpty(meta)) writer.writeInt(0); @@ -58,13 +60,14 @@ public class JdbcMetaTablesResult extends JdbcResult { writer.writeInt(meta.size()); for(JdbcTableMeta m : meta) - m.writeBinary(writer); + m.writeBinary(writer, ver); } } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); int size = reader.readInt(); @@ -76,7 +79,7 @@ public class JdbcMetaTablesResult extends JdbcResult { for (int i = 0; i < size; ++i) { JdbcTableMeta m = new JdbcTableMeta(); - m.readBinary(reader); + m.readBinary(reader, ver); meta.add(m); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java index 1d4d4ac4838f2..743978d109b3e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.NotNull; @@ -61,15 +62,15 @@ public long order() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(order); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); order = reader.readLong(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteResult.java index 84853d484f3c2..76f665f46ccb5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteResult.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -54,16 +55,18 @@ public long order() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(order); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); order = reader.readLong(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java index c0cfc9e1d4a5a..52e5a19ee346d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java @@ -22,6 +22,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -133,7 +134,8 @@ public int mode() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { writer.writeInt(isNullable); writer.writeBoolean(signed); writer.writeInt(precision); @@ -145,7 +147,8 @@ public int mode() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { isNullable = reader.readInt(); signed = reader.readBoolean(); precision = reader.readInt(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java index 6b9bf70f9fcca..dffbdca0f4d0b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; /** @@ -89,7 +90,8 @@ public List fields() { /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { writer.writeString(schemaName); writer.writeString(tblName); writer.writeString(name); @@ -98,7 +100,8 @@ public List fields() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { schemaName = reader.readString(); tblName = reader.readString(); name = reader.readString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQuery.java index f7ffb994937c9..d9960dabe2fa0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQuery.java @@ -19,6 +19,7 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.SqlListenerUtils; import org.apache.ignite.internal.util.typedef.internal.S; @@ -63,7 +64,8 @@ public Object[] args() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) { writer.writeString(sql); if (args == null || args.length == 0) @@ -77,7 +79,8 @@ public Object[] args() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) { sql = reader.readString(); int argsNum = reader.readInt(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java index 872889ca9b775..5c631c33d31de 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -52,15 +53,17 @@ public long queryId() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(queryId); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); queryId = reader.readLong(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java index 9bbdd593bfa75..44a56aa2bdb31 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java @@ -23,6 +23,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -80,14 +81,15 @@ public boolean isLast() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); if (results != null && results.size() > 0) { writer.writeInt(results.size()); for (JdbcResultInfo r : results) - r.writeBinary(writer); + r.writeBinary(writer, ver); if (results.get(0).isQuery()) { writer.writeBoolean(last); @@ -101,8 +103,9 @@ public boolean isLast() { /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); int cnt = reader.readInt(); @@ -114,7 +117,7 @@ public boolean isLast() { for (int i = 0; i < cnt; ++i) { JdbcResultInfo r = new JdbcResultInfo(); - r.readBinary(reader); + r.readBinary(reader, ver); results.add(r); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java index c13117a2dfbac..e4f3398b09cf4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java @@ -21,12 +21,15 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.SqlListenerUtils; 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.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_7_0; + /** * JDBC query execute request. */ @@ -58,6 +61,8 @@ public class JdbcQueryExecuteRequest extends JdbcRequest { */ JdbcQueryExecuteRequest() { super(QRY_EXEC); + + autoCommit = true; } /** @@ -132,8 +137,9 @@ boolean autoCommit() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeString(schemaName); writer.writeInt(pageSize); @@ -147,15 +153,17 @@ boolean autoCommit() { SqlListenerUtils.writeObject(writer, arg, false); } - writer.writeByte((byte)stmtType.ordinal()); + if (ver.compareTo(VER_2_7_0) >= 0) + writer.writeBoolean(autoCommit); - writer.writeBoolean(autoCommit); + writer.writeByte((byte)stmtType.ordinal()); } /** {@inheritDoc} */ @SuppressWarnings("SimplifiableIfStatement") - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); schemaName = reader.readString(); pageSize = reader.readInt(); @@ -169,6 +177,9 @@ boolean autoCommit() { for (int i = 0; i < argsNum; ++i) args[i] = SqlListenerUtils.readObject(reader, false); + if (ver.compareTo(VER_2_7_0) >= 0) + autoCommit = reader.readBoolean(); + try { if (reader.available() > 0) stmtType = JdbcStatementType.fromOrdinal(reader.readByte()); @@ -178,16 +189,6 @@ boolean autoCommit() { catch (IOException e) { throw new BinaryObjectException(e); } - - try { - if (reader.available() > 0) - autoCommit = reader.readBoolean(); - else - autoCommit = true; - } - catch (IOException e) { - throw new BinaryObjectException(e); - } } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java index fdebdb8c3c450..342e8ef9b63a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -112,8 +113,9 @@ public long updateCount() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(queryId); writer.writeBoolean(isQuery); @@ -131,8 +133,9 @@ public long updateCount() { /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); queryId = reader.readLong(); isQuery = reader.readBoolean(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java index 776c3bfa85438..59ed9a87cf17a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -65,16 +66,18 @@ public int pageSize() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(queryId); writer.writeInt(pageSize); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); queryId = reader.readLong(); pageSize = reader.readInt(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java index ac4a6035e6ab6..e62efcb198f19 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -66,8 +67,9 @@ public boolean last() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeBoolean(last); @@ -75,8 +77,9 @@ public boolean last() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); last = reader.readBoolean(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java index bdef321abe678..f30ecfd0fd625 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -53,15 +54,17 @@ public long queryId() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); writer.writeLong(qryId); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); qryId = reader.readLong(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java index c8c0991e344bb..d366e609cf86c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java @@ -23,6 +23,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -58,8 +59,9 @@ public List meta() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { - super.writeBinary(writer); + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.writeBinary(writer, ver); if (F.isEmpty(meta)) writer.writeInt(0); @@ -67,13 +69,14 @@ public List meta() { writer.writeInt(meta.size()); for (JdbcColumnMeta m : meta) - m.writeBinary(writer); + m.writeBinary(writer, ver); } } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { - super.readBinary(reader); + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { + super.readBinary(reader, ver); int size = reader.readInt(); @@ -85,7 +88,7 @@ public List meta() { for (int i = 0; i < size; ++i) { JdbcColumnMeta m = new JdbcColumnMeta(); - m.readBinary(reader); + m.readBinary(reader, ver); meta.add(m); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRawBinarylizable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRawBinarylizable.java index c3f1874ceae46..22c514d9391e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRawBinarylizable.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRawBinarylizable.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; /** * Interface that allows to implement custom serialization @@ -30,15 +31,18 @@ public interface JdbcRawBinarylizable { * Writes fields to provided writer. * * @param writer Binary object writer. + * @param ver ver Protocol version. * @throws BinaryObjectException In case of error. */ - public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException; + public void writeBinary(BinaryWriterExImpl writer, ClientListenerProtocolVersion ver) throws BinaryObjectException; /** * Reads fields from provided reader. * * @param reader Binary object reader. + * @param ver Protocol version. * @throws BinaryObjectException In case of error. */ - public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException; + public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException; } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java index 3d5b86973282d..0674edfb65156 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.ClientListenerRequestNoId; /** @@ -77,12 +78,14 @@ public JdbcRequest(byte type) { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { writer.writeByte(type); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { // No-op. } @@ -95,10 +98,12 @@ public byte type() { /** * @param reader Binary reader. + * @param ver Protocol version. * @return Request object. * @throws BinaryObjectException On error. */ - public static JdbcRequest readRequest(BinaryReaderExImpl reader) throws BinaryObjectException { + public static JdbcRequest readRequest(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { int reqType = reader.readByte(); JdbcRequest req; @@ -173,7 +178,7 @@ public static JdbcRequest readRequest(BinaryReaderExImpl reader) throws BinaryOb throw new IgniteException("Unknown SQL listener request ID: [request ID=" + reqType + ']'); } - req.readBinary(reader); + req.readBinary(reader, ver); return req; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java index 91f26d8976066..5d5b4e38f625f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; @@ -76,14 +77,15 @@ public JdbcResult response() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { writer.writeInt(status()); if (status() == STATUS_SUCCESS) { writer.writeBoolean(res != null); if (res != null) - res.writeBinary(writer); + res.writeBinary(writer, ver); } else writer.writeString(error()); @@ -91,12 +93,13 @@ public JdbcResult response() { } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { status(reader.readInt()); if (status() == STATUS_SUCCESS) { if (reader.readBoolean()) - res = JdbcResult.readResult(reader); + res = JdbcResult.readResult(reader, ver); } else error(reader.readString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java index 199e5da1d14ef..3a0d7bbc9166d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; /** * JDBC response result. @@ -87,21 +88,24 @@ public JdbcResult(byte type) { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { writer.writeByte(type); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { // No-op. } /** * @param reader Binary reader. + * @param ver Protocol verssion. * @return Request object. * @throws BinaryObjectException On error. */ - public static JdbcResult readResult(BinaryReaderExImpl reader) throws BinaryObjectException { + public static JdbcResult readResult(BinaryReaderExImpl reader, ClientListenerProtocolVersion ver) throws BinaryObjectException { int resId = reader.readByte(); JdbcResult res; @@ -191,7 +195,7 @@ public static JdbcResult readResult(BinaryReaderExImpl reader) throws BinaryObje throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']'); } - res.readBinary(reader); + res.readBinary(reader, ver); return res; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java index f0706e4a40cb0..5fab77aa3f00b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResultInfo.java @@ -19,6 +19,7 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -75,14 +76,16 @@ public long updateCount() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) { writer.writeBoolean(isQuery); writer.writeLong(updCnt); writer.writeLong(qryId); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) { isQuery = reader.readBoolean(); updCnt = reader.readLong(); qryId = reader.readLong(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java index 5e1518949c426..d4324d679e22e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java @@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -66,13 +67,15 @@ public String tableName() { } /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + @Override public void writeBinary(BinaryWriterExImpl writer, + ClientListenerProtocolVersion ver) throws BinaryObjectException { writer.writeString(schemaName); writer.writeString(tblName); } /** {@inheritDoc} */ - @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + @Override public void readBinary(BinaryReaderExImpl reader, + ClientListenerProtocolVersion ver) throws BinaryObjectException { schemaName = reader.readString(); tblName = reader.readString(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java index 2b51741d9e93f..d82dcc695cd45 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java @@ -47,7 +47,7 @@ public class OdbcConnectionContext extends ClientListenerAbstractConnectionConte /** Version 2.3.2: added multiple statements support. */ public static final ClientListenerProtocolVersion VER_2_3_2 = ClientListenerProtocolVersion.create(2, 3, 2); - /** Version 2.5.0: added authentication and transactions. */ + /** Version 2.5.0: added authentication. */ public static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0); /** Version 2.7.0: added precision and scale. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java index 552841d20ddf9..3ed6f21c846f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java @@ -103,7 +103,7 @@ public OdbcMessageParser(GridKernalContext ctx, ClientListenerProtocolVersion ve boolean autoCommit = true; - if (ver.compareTo(OdbcConnectionContext.VER_2_5_0) >= 0) + if (ver.compareTo(OdbcConnectionContext.VER_2_7_0) >= 0) autoCommit = reader.readBoolean(); res = new OdbcQueryExecuteRequest(schema, sql, params, timeout, autoCommit); @@ -130,7 +130,7 @@ public OdbcMessageParser(GridKernalContext ctx, ClientListenerProtocolVersion ve boolean autoCommit = true; - if (ver.compareTo(OdbcConnectionContext.VER_2_5_0) >= 0) + if (ver.compareTo(OdbcConnectionContext.VER_2_7_0) >= 0) autoCommit = reader.readBoolean(); res = new OdbcQueryExecuteBatchRequest(schema, sql, last, params, timeout, autoCommit); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java index 86721949705c3..0bec66e56be42 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java @@ -476,67 +476,67 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { writer.incrementState(); case 2: - if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) + if (!writer.writeInt("pageSize", pageSize)) return false; writer.incrementState(); case 3: - if (!writer.writeInt("pageSize", pageSize)) + if (!writer.writeByteArray("paramsBytes", paramsBytes)) return false; writer.incrementState(); case 4: - if (!writer.writeByteArray("paramsBytes", paramsBytes)) + if (!writer.writeMap("parts", parts, MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR)) return false; writer.incrementState(); case 5: - if (!writer.writeMap("parts", parts, MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR)) + if (!writer.writeCollection("qrys", qrys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 6: - if (!writer.writeIntArray("qryParts", qryParts)) + if (!writer.writeLong("reqId", reqId)) return false; writer.incrementState(); case 7: - if (!writer.writeCollection("qrys", qrys, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("tbls", tbls, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 8: - if (!writer.writeLong("reqId", reqId)) + if (!writer.writeInt("timeout", timeout)) return false; writer.incrementState(); case 9: - if (!writer.writeString("schemaName", schemaName)) + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); case 10: - if (!writer.writeCollection("tbls", tbls, MessageCollectionItemType.MSG)) + if (!writer.writeIntArray("qryParts", qryParts)) return false; writer.incrementState(); case 11: - if (!writer.writeInt("timeout", timeout)) + if (!writer.writeString("schemaName", schemaName)) return false; writer.incrementState(); case 12: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeMessage("mvccSnapshot", mvccSnapshot)) return false; writer.incrementState(); @@ -577,7 +577,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 2: - mvccSnapshot = reader.readMessage("mvccSnapshot"); + pageSize = reader.readInt("pageSize"); if (!reader.isLastRead()) return false; @@ -585,7 +585,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 3: - pageSize = reader.readInt("pageSize"); + paramsBytes = reader.readByteArray("paramsBytes"); if (!reader.isLastRead()) return false; @@ -593,7 +593,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 4: - paramsBytes = reader.readByteArray("paramsBytes"); + parts = reader.readMap("parts", MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR, false); if (!reader.isLastRead()) return false; @@ -601,7 +601,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 5: - parts = reader.readMap("parts", MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR, false); + qrys = reader.readCollection("qrys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -609,7 +609,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 6: - qryParts = reader.readIntArray("qryParts"); + reqId = reader.readLong("reqId"); if (!reader.isLastRead()) return false; @@ -617,7 +617,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 7: - qrys = reader.readCollection("qrys", MessageCollectionItemType.MSG); + tbls = reader.readCollection("tbls", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -625,7 +625,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 8: - reqId = reader.readLong("reqId"); + timeout = reader.readInt("timeout"); if (!reader.isLastRead()) return false; @@ -633,7 +633,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 9: - schemaName = reader.readString("schemaName"); + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -641,7 +641,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 10: - tbls = reader.readCollection("tbls", MessageCollectionItemType.MSG); + qryParts = reader.readIntArray("qryParts"); if (!reader.isLastRead()) return false; @@ -649,7 +649,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 11: - timeout = reader.readInt("timeout"); + schemaName = reader.readString("schemaName"); if (!reader.isLastRead()) return false; @@ -657,7 +657,7 @@ public void txDetails(GridH2SelectForUpdateTxDetails txDetails) { reader.incrementState(); case 12: - topVer = reader.readMessage("topVer"); + mvccSnapshot = reader.readMessage("mvccSnapshot"); if (!reader.isLastRead()) return false; From 024ed9e49f71740532e82bcf5322574f0148b881 Mon Sep 17 00:00:00 2001 From: devozerov Date: Fri, 7 Sep 2018 12:19:55 +0300 Subject: [PATCH 62/95] IGNITE-8927: SQL: correct handling of LOST partitions on mapper side. Note that for local queries this doesn't work still because partition state is not checked (see IGNITE-7039). This closes #4679. --- .../messages/GridQueryFailResponse.java | 2 +- ...gniteCachePartitionLossPolicySelfTest.java | 29 +++- .../h2/twostep/GridMapQueryExecutor.java | 123 +++++++++++----- .../h2/twostep/GridReduceQueryExecutor.java | 2 +- ...exingCachePartitionLossPolicySelfTest.java | 139 ++++++++++++++++++ ...teCacheDistributedQueryCancelSelfTest.java | 2 +- .../h2/twostep/RetryCauseMessageSelfTest.java | 3 +- .../IgniteCacheQuerySelfTestSuite.java | 4 + 8 files changed, 261 insertions(+), 43 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IndexingCachePartitionLossPolicySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java index 1b759bbfa2e24..ef26d2a8e2988 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java @@ -59,7 +59,7 @@ public GridQueryFailResponse() { */ public GridQueryFailResponse(long qryReqId, Throwable err) { this.qryReqId = qryReqId; - this.errMsg = err.getClass() + ":" + err.getMessage(); + this.errMsg = err.getMessage(); this.failCode = err instanceof QueryCancelledException ? CANCELLED_BY_ORIGINATOR : GENERAL_ERROR; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java index 0be85260b6cbc..1aacc9c90f6f8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java @@ -72,7 +72,7 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe private PartitionLossPolicy partLossPlc; /** */ - private static final String CACHE_NAME = "partitioned"; + protected static final String CACHE_NAME = "partitioned"; /** */ private int backups = 0; @@ -101,6 +101,15 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe cfg.setClientMode(client); + cfg.setCacheConfiguration(cacheConfiguration()); + + return cfg; + } + + /** + * @return Cache configuration. + */ + protected CacheConfiguration cacheConfiguration() { CacheConfiguration cacheCfg = new CacheConfiguration<>(CACHE_NAME); cacheCfg.setCacheMode(PARTITIONED); @@ -109,9 +118,7 @@ public class IgniteCachePartitionLossPolicySelfTest extends GridCommonAbstractTe cacheCfg.setPartitionLossPolicy(partLossPlc); cacheCfg.setAffinity(new RendezvousAffinityFunction(false, 32)); - cfg.setCacheConfiguration(cacheCfg); - - return cfg; + return cacheCfg; } /** {@inheritDoc} */ @@ -294,6 +301,9 @@ private void checkLostPartition(boolean canWrite, boolean safe, TopologyChanger // Check that writing in recover mode does not clear partition state. verifyCacheOps(canWrite, safe, part, ig); + + // Validate queries. + validateQuery(safe, part, ig); } // Check that partition state does not change after we start a new node. @@ -410,6 +420,17 @@ protected List noPrimaryOrBackupPartition(List nodes) { return parts; } + /** + * Validate query execution on a node. + * + * @param safe Safe flag. + * @param part Partition. + * @param node Node. + */ + protected void validateQuery(boolean safe, int part, Ignite node) { + // No-op. + } + /** */ class TopologyChanger { /** Flag to delay partition exchange */ 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 39c12c65792da..baa5f9c074dda 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 @@ -41,6 +41,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.PartitionLossPolicy; import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cluster.ClusterNode; @@ -54,6 +55,7 @@ import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheInvalidStateException; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.distributed.dht.CompoundLockFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; @@ -102,9 +104,12 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_FORCE_LAZY_RESULT_SET; +import static org.apache.ignite.cache.PartitionLossPolicy.READ_ONLY_SAFE; +import static org.apache.ignite.cache.PartitionLossPolicy.READ_WRITE_SAFE; import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.QUERY_POOL; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.LOST; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF; import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.distributedJoinMode; @@ -335,11 +340,9 @@ private String reservePartitions( // Cache was not found, probably was not deployed yet. if (cctx == null) { - final String res = String.format("Failed to reserve partitions for query (cache is not found on " + + return String.format("Failed to reserve partitions for query (cache is not found on " + "local node) [localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s]", ctx.localNodeId(), nodeId, reqId, topVer, cacheIds.get(i)); - - return res; } if (cctx.isLocal() || !cctx.rebalanceEnabled()) @@ -396,16 +399,39 @@ private String reservePartitions( if (explicitParts == null) partIds = cctx.affinity().primaryPartitions(ctx.localNodeId(), topVer); + int reservedCnt = 0; + for (int partId : partIds) { GridDhtLocalPartition part = partition(cctx, partId); GridDhtPartitionState partState = part != null ? part.state() : null; - if (partState != OWNING || !part.reserve()) + if (partState != OWNING) { + if (partState == LOST) + ignoreLostPartitionIfPossible(cctx, part); + else { + return String.format("Failed to reserve partitions for query " + + "(partition of PARTITIONED cache is not found or not in OWNING state) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, " + + "cacheName=%s, part=%s, partFound=%s, partState=%s]", + ctx.localNodeId(), + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + partId, + (part != null), + partState + ); + } + } + + if (!part.reserve()) { return String.format("Failed to reserve partitions for query " + - "(partition of PARTITIONED cache cannot be reserved) [" + - "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + - "part=%s, partFound=%s, partState=%s]", + "(partition of PARTITIONED cache cannot be reserved) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, " + + "cacheName=%s, part=%s, partFound=%s, partState=%s]", ctx.localNodeId(), nodeId, reqId, @@ -413,36 +439,44 @@ private String reservePartitions( cacheIds.get(i), cctx.name(), partId, - (part != null), + true, partState ); + } reserved.add(part); + reservedCnt++; + // Double check that we are still in owning state and partition contents are not cleared. partState = part.state(); - if (part.state() != OWNING) - return String.format("Failed to reserve partitions for query " + - "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + - "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, cacheName=%s, " + - "part=%s, partState=%s]", - ctx.localNodeId(), - nodeId, - reqId, - topVer, - cacheIds.get(i), - cctx.name(), - partId, - partState - ); + if (partState != OWNING) { + if (partState == LOST) + ignoreLostPartitionIfPossible(cctx, part); + else { + return String.format("Failed to reserve partitions for query " + + "(partition of PARTITIONED cache is not in OWNING state after reservation) [" + + "localNodeId=%s, rmtNodeId=%s, reqId=%s, affTopVer=%s, cacheId=%s, " + + "cacheName=%s, part=%s, partState=%s]", + ctx.localNodeId(), + nodeId, + reqId, + topVer, + cacheIds.get(i), + cctx.name(), + partId, + partState + ); + } + } } - if (explicitParts == null) { + if (explicitParts == null && reservedCnt > 0) { // We reserved all the primary partitions for cache, attempt to add group reservation. GridDhtPartitionsReservation grp = new GridDhtPartitionsReservation(topVer, cctx, "SQL"); - if (grp.register(reserved.subList(reserved.size() - partIds.size(), reserved.size()))) { + if (grp.register(reserved.subList(reserved.size() - reservedCnt, reserved.size()))) { if (reservations.putIfAbsent(grpKey, grp) != null) throw new IllegalStateException("Reservation already exists."); @@ -460,6 +494,25 @@ private String reservePartitions( return null; } + /** + * Decide whether to ignore or proceed with lost partition. + * + * @param cctx Cache context. + * @param part Partition. + * @throws IgniteCheckedException If failed. + */ + private static void ignoreLostPartitionIfPossible(GridCacheContext cctx, GridDhtLocalPartition part) + throws IgniteCheckedException { + PartitionLossPolicy plc = cctx.config().getPartitionLossPolicy(); + + if (plc != null) { + if (plc == READ_ONLY_SAFE || plc == READ_WRITE_SAFE) { + throw new CacheInvalidStateException("Failed to execute query because cache partition has been " + + "lost [cacheName=" + cctx.name() + ", part=" + part + ']'); + } + } + } + /** * @param ints Integers. * @return Collection wrapper. @@ -1054,22 +1107,22 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th List reserved = new ArrayList<>(); - String err = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); + MapNodeResults nodeResults = resultsForNode(node.id()); - if (!F.isEmpty(err)) { - U.error(log, "Failed to reserve partitions for DML request. [localNodeId=" + ctx.localNodeId() + - ", nodeId=" + node.id() + ", reqId=" + req.requestId() + ", cacheIds=" + cacheIds + - ", topVer=" + topVer + ", parts=" + Arrays.toString(parts) + ']'); + try { + String err = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); - sendUpdateResponse(node, reqId, null, - "Failed to reserve partitions for DML request. " + err); + if (!F.isEmpty(err)) { + U.error(log, "Failed to reserve partitions for DML request. [localNodeId=" + ctx.localNodeId() + + ", nodeId=" + node.id() + ", reqId=" + req.requestId() + ", cacheIds=" + cacheIds + + ", topVer=" + topVer + ", parts=" + Arrays.toString(parts) + ']'); - return; - } + sendUpdateResponse(node, reqId, null, + "Failed to reserve partitions for DML request. " + err); - MapNodeResults nodeResults = resultsForNode(node.id()); + return; + } - try { IndexingQueryFilter filter = h2.backupFilter(topVer, parts); GridQueryCancel cancel = nodeResults.putUpdate(reqId); 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 586fb5118ef47..96c88ffe7f4fb 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 @@ -884,7 +884,7 @@ else if (mvccTracker != null) if (wasCancelled(err)) throw new QueryCancelledException(); // Throw correct exception. - throw new CacheException("Failed to run map query remotely: " + err.getMessage(), err); + throw err; } else { retry = true; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IndexingCachePartitionLossPolicySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IndexingCachePartitionLossPolicySelfTest.java new file mode 100644 index 0000000000000..f2085994b0cd0 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IndexingCachePartitionLossPolicySelfTest.java @@ -0,0 +1,139 @@ +/* + * 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.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.cache.distributed.IgniteCachePartitionLossPolicySelfTest; + +import java.util.Collection; + +/** + * Partition loss policy test with enabled indexing. + */ +public class IndexingCachePartitionLossPolicySelfTest extends IgniteCachePartitionLossPolicySelfTest { + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration() { + CacheConfiguration ccfg = super.cacheConfiguration(); + + ccfg.setIndexedTypes(Integer.class, Integer.class); + + return ccfg; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected void validateQuery(boolean safe, int part, Ignite node) { + // Get node lost and remaining partitions. + IgniteCache cache = node.cache(CACHE_NAME); + + Collection lostParts = cache.lostPartitions(); + + Integer remainingPart = null; + + for (int i = 0; i < node.affinity(CACHE_NAME).partitions(); i++) { + if (lostParts.contains(i)) + continue; + + remainingPart = i; + + break; + } + + // Determine whether local query should be executed on that node. + boolean execLocQry = false; + + for (int nodePrimaryPart : node.affinity(CACHE_NAME).primaryPartitions(node.cluster().localNode())) { + if (part == nodePrimaryPart) { + execLocQry = true; + + break; + } + } + + // 1. Check query against all partitions. + validateQuery0(safe, node, false); + + // TODO: https://issues.apache.org/jira/browse/IGNITE-7039 +// if (execLocQry) +// validateQuery0(safe, node, true); + + // 2. Check query against LOST partition. + validateQuery0(safe, node, false, part); + + // TODO: https://issues.apache.org/jira/browse/IGNITE-7039 +// if (execLocQry) +// validateQuery0(safe, node, true, part); + + // 3. Check query on remaining partition. + if (remainingPart != null) { + executeQuery(node, false, remainingPart); + + // 4. Check query over two partitions - normal and LOST. + validateQuery0(safe, node, false, part, remainingPart); + } + } + + /** + * Query validation routine. + * + * @param safe Safe flag. + * @param node Node. + * @param loc Local flag. + * @param parts Partitions. + */ + private void validateQuery0(boolean safe, Ignite node, boolean loc, int... parts) { + if (safe) { + try { + executeQuery(node, loc, parts); + + fail("Exception is not thrown."); + } + catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage() != null && + e.getMessage().contains("Failed to execute query because cache partition has been lost")); + } + } + else { + executeQuery(node, loc, parts); + } + } + + /** + * Execute SQL query on a given node. + * + * @param parts Partitions. + * @param node Node. + * @param loc Local flag. + */ + private static void executeQuery(Ignite node, boolean loc, int... parts) { + IgniteCache cache = node.cache(CACHE_NAME); + + SqlFieldsQuery qry = new SqlFieldsQuery("SELECT * FROM Integer"); + + if (parts != null && parts.length != 0) + qry.setPartitions(parts); + + if (loc) + qry.setLocal(true); + + cache.query(qry).getAll(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java index e26b2111f90c5..d5ee0e978f874 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java @@ -158,7 +158,7 @@ public void testQueryResponseFailCode() throws Exception { fail(); } catch (Exception e) { - assertTrue(e.getCause() instanceof CacheException); + assertTrue(e instanceof CacheException); } } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java index 326988739ec57..ce385114485a7 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java @@ -250,7 +250,8 @@ public void testPartitionedCacheReserveFailureMessage() { personCache.query(qry).getAll(); } catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) [")); + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED " + + "cache is not found or not in OWNING state) ")); return; } 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 671db13178b70..7f67b3575209b 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 @@ -77,6 +77,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCrossCachesJoinsQueryTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicSqlRestoreTest; import org.apache.ignite.internal.processors.cache.IncorrectQueryEntityTest; +import org.apache.ignite.internal.processors.cache.IndexingCachePartitionLossPolicySelfTest; import org.apache.ignite.internal.processors.cache.QueryEntityCaseMismatchTest; import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.authentication.SqlUserCommandSelfTest; @@ -469,6 +470,9 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(H2StatementCacheSelfTest.class); suite.addTestSuite(PreparedStatementExSelfTest.class); + // Partition loss. + suite.addTestSuite(IndexingCachePartitionLossPolicySelfTest.class); + return suite; } } From 558218ff4060a3eb0e13ccdcef0afbd1af91000d Mon Sep 17 00:00:00 2001 From: zaleslaw Date: Fri, 7 Sep 2018 13:31:58 +0300 Subject: [PATCH 63/95] IGNITE-9482: [ML] Refactor all trainers' settters to withFieldName format for meta-algorithms this closes #4699 --- .../KMeansClusterizationExample.java | 4 +- .../RandomForestClassificationExample.java | 2 +- .../RandomForestRegressionExample.java | 2 +- .../kmeans/ClusterizationModel.java | 4 +- .../ml/clustering/kmeans/KMeansModel.java | 4 +- .../ml/clustering/kmeans/KMeansTrainer.java | 8 +- .../ml/knn/ann/ANNClassificationTrainer.java | 4 +- .../org/apache/ignite/ml/nn/MLPTrainer.java | 165 +++++++++++++++++- .../BinarizationPreprocessor.java | 4 +- .../binarization/BinarizationTrainer.java | 9 +- .../encoding/EncoderTrainer.java | 11 ++ .../linear/LinearRegressionSGDTrainer.java | 58 +++++- .../LogisticRegressionSGDTrainer.java | 107 +++++++++++- .../LogRegressionMultiClassTrainer.java | 26 +-- .../ml/selection/cv/CrossValidation.java | 2 +- .../ml/selection/paramgrid/ParamGrid.java | 14 +- .../SVMLinearBinaryClassificationTrainer.java | 22 +-- ...LinearMultiClassClassificationTrainer.java | 22 +-- .../apache/ignite/ml/tree/DecisionTree.java | 4 +- .../DecisionTreeClassificationTrainer.java | 4 +- .../tree/DecisionTreeRegressionTrainer.java | 10 +- .../GDBBinaryClassifierOnTreesTrainer.java | 73 ++++++-- .../boosting/GDBRegressionOnTreesTrainer.java | 67 ++++++- .../randomforest/RandomForestTrainer.java | 14 +- .../ignite/ml/clustering/KMeansModelTest.java | 4 +- .../ml/clustering/KMeansTrainerTest.java | 8 +- .../ignite/ml/common/LocalModelsTest.java | 2 +- .../composition/boosting/GDBTrainerTest.java | 8 +- .../environment/LearningEnvironmentTest.java | 2 +- .../binarization/BinarizationTrainerTest.java | 6 +- .../logistic/LogRegMultiClassTrainerTest.java | 8 +- .../DecisionTreeRegressionTrainerTest.java | 2 +- .../tree/randomforest/RandomForestTest.java | 2 +- 33 files changed, 540 insertions(+), 142 deletions(-) diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java index b96cbce9e55a2..152375ac7d912 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java @@ -70,8 +70,8 @@ public static void main(String[] args) throws InterruptedException { ); System.out.println(">>> KMeans centroids"); - Tracer.showAscii(mdl.centers()[0]); - Tracer.showAscii(mdl.centers()[1]); + Tracer.showAscii(mdl.getCenters()[0]); + Tracer.showAscii(mdl.getCenters()[1]); System.out.println(">>>"); System.out.println(">>> -----------------------------------"); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java index aa13943f58f08..6194153d11077 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestClassificationExample.java @@ -74,7 +74,7 @@ public static void main(String[] args) throws InterruptedException { .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) .withMaxDepth(4) .withMinImpurityDelta(0.) - .withSubsampleSize(0.3) + .withSubSampleSize(0.3) .withSeed(0); System.out.println(">>> Configured trainer: " + classifier.getClass().getSimpleName()); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java index e2bfe8b84b876..5f010f2a6332c 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/randomforest/RandomForestRegressionExample.java @@ -78,7 +78,7 @@ public static void main(String[] args) throws InterruptedException { .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) .withMaxDepth(4) .withMinImpurityDelta(0.) - .withSubsampleSize(0.3) + .withSubSampleSize(0.3) .withSeed(0); trainer.setEnvironment(LearningEnvironment.builder() diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/ClusterizationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/ClusterizationModel.java index 474a463eb7550..43e189905e21c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/ClusterizationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/ClusterizationModel.java @@ -22,8 +22,8 @@ /** Base interface for all clusterization models. */ public interface ClusterizationModel extends Model { /** Gets the clusters count. */ - public int amountOfClusters(); + public int getAmountOfClusters(); /** Get cluster centers. */ - public P[] centers(); + public P[] getCenters(); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModel.java index bdfa1b61598e7..e07f4f00ac50e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModel.java @@ -54,12 +54,12 @@ public DistanceMeasure distanceMeasure() { } /** Amount of centers in clusterization. */ - @Override public int amountOfClusters() { + @Override public int getAmountOfClusters() { return centers.length; } /** Get centers of clusters. */ - @Override public Vector[] centers() { + @Override public Vector[] getCenters() { return Arrays.copyOf(centers, centers.length); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java index 2596dbcfb9121..a20d5da39da3e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java @@ -106,7 +106,7 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { return getLastTrainedModelOrThrowEmptyDatasetException(mdl); centers = Optional.ofNullable(mdl) - .map(KMeansModel::centers) + .map(KMeansModel::getCenters) .orElseGet(() -> initClusterCentersRandomly(dataset, k)); boolean converged = false; @@ -143,7 +143,7 @@ public class KMeansTrainer extends SingleLabelDatasetTrainer { /** {@inheritDoc} */ @Override protected boolean checkState(KMeansModel mdl) { - return mdl.centers().length == k && mdl.distanceMeasure().equals(distance); + return mdl.getCenters().length == k && mdl.distanceMeasure().equals(distance); } /** @@ -313,7 +313,7 @@ public ConcurrentHashMap> getCentroi * * @return The parameter value. */ - public int getK() { + public int getAmountOfClusters() { return k; } @@ -323,7 +323,7 @@ public int getK() { * @param k The parameter value. * @return Model with new amount of clusters parameter value. */ - public KMeansTrainer withK(int k) { + public KMeansTrainer withAmountOfClusters(int k) { this.k = k; return this; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java index 3e32b6700310c..e56a10ae56be5 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/ann/ANNClassificationTrainer.java @@ -131,7 +131,7 @@ public class ANNClassificationTrainer extends SingleLabelDatasetTrainer List getCentroids(IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, DatasetBuilder datasetBuilder) { KMeansTrainer trainer = new KMeansTrainer() - .withK(k) + .withAmountOfClusters(k) .withMaxIterations(maxIterations) .withSeed(seed) .withDistance(distance) @@ -143,7 +143,7 @@ private List getCentroids(IgniteBiFunction featureE lbExtractor ); - return Arrays.asList(mdl.centers()); + return Arrays.asList(mdl.getCenters()); } /** */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java index 8f1a4cb8e8159..1cac909e8a27f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java @@ -46,25 +46,25 @@ */ public class MLPTrainer

      extends MultiLabelDatasetTrainer { /** Multilayer perceptron architecture supplier that defines layers and activators. */ - private final IgniteFunction, MLPArchitecture> archSupplier; + private IgniteFunction, MLPArchitecture> archSupplier; /** Loss function to be minimized during the training. */ - private final IgniteFunction loss; + private IgniteFunction loss; /** Update strategy that defines how to update model parameters during the training. */ - private final UpdatesStrategy updatesStgy; + private UpdatesStrategy updatesStgy; /** Maximal number of iterations before the training will be stopped. */ - private final int maxIterations; + private int maxIterations = 100; /** Batch size (per every partition). */ - private final int batchSize; + private int batchSize = 100; /** Maximal number of local iterations before synchronization. */ - private final int locIterations; + private int locIterations = 100; /** Multilayer perceptron model initializer. */ - private final long seed; + private long seed = 1234L; /** * Constructs a new instance of multilayer perceptron trainer. @@ -119,14 +119,18 @@ public MultilayerPerceptron fit(DatasetBuilder datasetBuilder, DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + assert archSupplier != null; + assert loss!= null; + assert updatesStgy!= null; + try (Dataset dataset = datasetBuilder.build( new EmptyContextBuilder<>(), new SimpleLabeledDatasetDataBuilder<>(featureExtractor, lbExtractor) )) { MultilayerPerceptron mdl; - if (lastLearnedModel != null) { + if (lastLearnedModel != null) mdl = lastLearnedModel; - } else { + else { MLPArchitecture arch = archSupplier.apply(dataset); mdl = new MultilayerPerceptron(arch, new RandomInitializer(seed)); } @@ -205,6 +209,149 @@ else if (b == null) } } + /** + * Get the multilayer perceptron architecture supplier that defines layers and activators. + * + * @return The property value. + */ + public IgniteFunction, MLPArchitecture> getArchSupplier() { + return archSupplier; + } + + /** + * Set up the multilayer perceptron architecture supplier that defines layers and activators. + * + * @param archSupplier The parameter value. + * @return Model with the multilayer perceptron architecture supplier that defines layers and activators. + */ + public MLPTrainer

      withArchSupplier( + IgniteFunction, MLPArchitecture> archSupplier) { + this.archSupplier = archSupplier; + return this; + } + + /** + * Get the loss function to be minimized during the training. + * + * @return The property value. + */ + public IgniteFunction getLoss() { + return loss; + } + + /** + * Set up the loss function to be minimized during the training. + * + * @param loss The parameter value. + * @return Model with the loss function to be minimized during the training. + */ + public MLPTrainer

      withLoss( + IgniteFunction loss) { + this.loss = loss; + return this; + } + + /** + * Get the update strategy that defines how to update model parameters during the training. + * + * @return The property value. + */ + public UpdatesStrategy getUpdatesStgy() { + return updatesStgy; + } + + /** + * Set up the update strategy that defines how to update model parameters during the training. + * + * @param updatesStgy The parameter value. + * @return Model with the update strategy that defines how to update model parameters during the training. + */ + public MLPTrainer

      withUpdatesStgy( + UpdatesStrategy updatesStgy) { + this.updatesStgy = updatesStgy; + return this; + } + + /** + * Get the maximal number of iterations before the training will be stopped. + * + * @return The property value. + */ + public int getMaxIterations() { + return maxIterations; + } + + /** + * Set up the maximal number of iterations before the training will be stopped. + * + * @param maxIterations The parameter value. + * @return Model with the maximal number of iterations before the training will be stopped. + */ + public MLPTrainer

      withMaxIterations(int maxIterations) { + this.maxIterations = maxIterations; + return this; + } + + /** + * Get the batch size (per every partition). + * + * @return The property value. + */ + public int getBatchSize() { + return batchSize; + } + + /** + * Set up the batch size (per every partition). + * + * @param batchSize The parameter value. + * @return Model with the batch size (per every partition). + */ + public MLPTrainer

      withBatchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + /** + * Get the maximal number of local iterations before synchronization. + * + * @return The property value. + */ + public int getLocIterations() { + return locIterations; + } + + /** + * Set up the maximal number of local iterations before synchronization. + * + * @param locIterations The parameter value. + * @return Model with the maximal number of local iterations before synchronization. + */ + public MLPTrainer

      withLocIterations(int locIterations) { + this.locIterations = locIterations; + return this; + } + + /** + * Get the multilayer perceptron model initializer. + * + * @return The property value. + */ + public long getSeed() { + return seed; + } + + /** + * Set up the multilayer perceptron model initializer. + * + * @param seed The parameter value. + * @return Model with the multilayer perceptron model initializer. + */ + public MLPTrainer

      withSeed(long seed) { + this.seed = seed; + return this; + } + /** {@inheritDoc} */ @Override protected boolean checkState(MultilayerPerceptron mdl) { return true; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationPreprocessor.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationPreprocessor.java index 830082063042c..2e1bd5c0a466e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationPreprocessor.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationPreprocessor.java @@ -68,8 +68,8 @@ public BinarizationPreprocessor(double threshold, IgniteBiFunction return res; } - /** Gets the threshold parameter. */ - public double threshold() { + /** Get the threshold parameter. */ + public double getThreshold() { return threshold; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainer.java index 26541e07858ea..ad8c90e7f5c0d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainer.java @@ -39,15 +39,16 @@ public class BinarizationTrainer implements PreprocessingTrainer withEncoderType(EncoderType type) { this.encoderType = type; return this; } + + /** + * Sets the indices of features which should be encoded. + * + * @param handledIndices Indices of features which should be encoded. + * @return The changed trainer. + */ + public EncoderTrainer withEncoderType(Set handledIndices) { + this.handledIndices = handledIndices; + return this; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java index 125ed242459b5..4132d359047b9 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java @@ -44,17 +44,17 @@ public class LinearRegressionSGDTrainer

      extends SingleLa /** Update strategy. */ private final UpdatesStrategy updatesStgy; - /** Max number of iteration. */ + /** Max amount of iterations. */ private int maxIterations = 1000; /** Batch size. */ private int batchSize = 10; - /** Number of local iterations. */ + /** Amount of local iterations. */ private int locIterations = 100; /** Seed for random generator. */ - private long seed = System.currentTimeMillis(); + private long seed = 1234L; /** * Constructs a new instance of linear regression SGD trainer. @@ -89,9 +89,12 @@ public LinearRegressionSGDTrainer(UpdatesStrategy LinearRegressionModel updateModel(LinearRegressionModel mdl, DatasetBuilder datasetBuilder, + @Override protected LinearRegressionModel updateModel(LinearRegressionModel mdl, + DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + assert updatesStgy != null; + IgniteFunction, MLPArchitecture> archSupplier = dataset -> { int cols = dataset.compute(data -> { @@ -162,7 +165,7 @@ public LinearRegressionSGDTrainer(UpdatesStrategy withSeed(long seed) { this.seed = seed; return this; } + + /** + * Get the update strategy. + * + * @return The property value. + */ + public UpdatesStrategy getUpdatesStgy() { + return updatesStgy; + } + + /** + * Get the max amount of iterations. + * + * @return The property value. + */ + public int getMaxIterations() { + return maxIterations; + } + + /** + * Get the batch size. + * + * @return The property value. + */ + public int getBatchSize() { + return batchSize; + } + + /** + * Get the amount of local iterations. + * + * @return The property value. + */ + public int getLocIterations() { + return locIterations; + } + + /** + * Get the seed for random generator. + * + * @return The property value. + */ + public long getSeed() { + return seed; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java index 839dab5582007..fb5d5a0db09cf 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/binomial/LogisticRegressionSGDTrainer.java @@ -41,19 +41,19 @@ */ public class LogisticRegressionSGDTrainer

      extends SingleLabelDatasetTrainer { /** Update strategy. */ - private final UpdatesStrategy updatesStgy; + private UpdatesStrategy updatesStgy; /** Max number of iteration. */ - private final int maxIterations; + private int maxIterations; /** Batch size. */ - private final int batchSize; + private int batchSize; /** Number of local iterations. */ - private final int locIterations; + private int locIterations; /** Seed for random generator. */ - private final long seed; + private long seed; /** * Constructs a new instance of linear regression SGD trainer. @@ -116,10 +116,11 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy lbExtractorWrapper = (k, v) -> new double[] {lbExtractor.apply(k, v)}; MultilayerPerceptron mlp; - if(mdl != null) { + if (mdl != null) { mlp = restoreMLPState(mdl); mlp = trainer.update(mlp, datasetBuilder, featureExtractor, lbExtractorWrapper); - } else + } + else mlp = trainer.fit(datasetBuilder, featureExtractor, lbExtractorWrapper); double[] params = mlp.parameters().getStorage().data(); @@ -136,8 +137,10 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy mlpState.set(ith.index(), ith.get())); mlpState.set(mlpState.size() - 1, intercept); perceptron.setParameters(mlpState); + return perceptron; } @@ -152,4 +156,93 @@ public LogisticRegressionSGDTrainer(UpdatesStrategy withMaxIterations(int maxIterations) { + this.maxIterations = maxIterations; + return this; + } + + /** + * Set up the batchSize parameter. + * + * @param batchSize The size of learning batch. + * @return Trainer with new batch size parameter value. + */ + public LogisticRegressionSGDTrainer

      withBatchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + /** + * Set up the amount of local iterations of SGD algorithm. + * + * @param amountOfLocIterations The parameter value. + * @return Trainer with new locIterations parameter value. + */ + public LogisticRegressionSGDTrainer

      withLocIterations(int amountOfLocIterations) { + this.locIterations = amountOfLocIterations; + return this; + } + + /** + * Set up the random seed parameter. + * + * @param seed Seed for random generator. + * @return Trainer with new seed parameter value. + */ + public LogisticRegressionSGDTrainer

      withSeed(long seed) { + this.seed = seed; + return this; + } + + /** + * Get the update strategy. + * + * @return The property value. + */ + public UpdatesStrategy getUpdatesStgy() { + return updatesStgy; + } + + /** + * Get the max amount of iterations. + * + * @return The property value. + */ + public int getMaxIterations() { + return maxIterations; + } + + /** + * Get the batch size. + * + * @return The property value. + */ + public int getBatchSize() { + return batchSize; + } + + /** + * Get the amount of local iterations. + * + * @return The property value. + */ + public int getLocIterations() { + return locIterations; + } + + /** + * Get the seed for random generator. + * + * @return The property value. + */ + public long getSeed() { + return seed; + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java index eb44301e4ecd7..b9cdcc7bf9d49 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/logistic/multiclass/LogRegressionMultiClassTrainer.java @@ -77,14 +77,14 @@ public class LogRegressionMultiClassTrainer

      } /** {@inheritDoc} */ - @Override public LogRegressionMultiClassModel updateModel(LogRegressionMultiClassModel mdl, + @Override public LogRegressionMultiClassModel updateModel(LogRegressionMultiClassModel newMdl, DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { List classes = extractClassLabels(datasetBuilder, lbExtractor); if(classes.isEmpty()) - return getLastTrainedModelOrThrowEmptyDatasetException(mdl); + return getLastTrainedModelOrThrowEmptyDatasetException(newMdl); LogRegressionMultiClassModel multiClsMdl = new LogRegressionMultiClassModel(); @@ -101,12 +101,12 @@ public class LogRegressionMultiClassTrainer

      return 0.0; }; - LogisticRegressionModel model = Optional.ofNullable(mdl) + LogisticRegressionModel mdl = Optional.ofNullable(newMdl) .flatMap(multiClassModel -> multiClassModel.getModel(clsLb)) .map(learnedModel -> trainer.update(learnedModel, datasetBuilder, featureExtractor, lbTransformer)) .orElseGet(() -> trainer.fit(datasetBuilder, featureExtractor, lbTransformer)); - multiClsMdl.add(clsLb, model); + multiClsMdl.add(clsLb, mdl); }); return multiClsMdl; @@ -169,20 +169,20 @@ public LogRegressionMultiClassTrainer withBatchSize(int batchSize) { } /** - * Gets the batch size. + * Get the batch size. * * @return The parameter value. */ - public double batchSize() { + public double getBatchSize() { return batchSize; } /** - * Gets the amount of outer iterations of SGD algorithm. + * Get the amount of outer iterations of SGD algorithm. * * @return The parameter value. */ - public int amountOfIterations() { + public int getAmountOfIterations() { return amountOfIterations; } @@ -198,11 +198,11 @@ public LogRegressionMultiClassTrainer withAmountOfIterations(int amountOfIterati } /** - * Gets the amount of local iterations. + * Get the amount of local iterations. * * @return The parameter value. */ - public int amountOfLocIterations() { + public int getAmountOfLocIterations() { return amountOfLocIterations; } @@ -229,7 +229,7 @@ public LogRegressionMultiClassTrainer withSeed(long seed) { } /** - * Gets the seed for random generator. + * Get the seed for random generator. * * @return The parameter value. */ @@ -249,11 +249,11 @@ public LogRegressionMultiClassTrainer withUpdatesStgy(UpdatesStrategy updatesStg } /** - * Gets the update strategy.. + * Get the update strategy.. * * @return The parameter value. */ - public UpdatesStrategy updatesStgy() { + public UpdatesStrategy getUpdatesStgy() { return updatesStgy; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/selection/cv/CrossValidation.java b/modules/ml/src/main/java/org/apache/ignite/ml/selection/cv/CrossValidation.java index 1ade876c1f7fe..ef4f30f06ec76 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/selection/cv/CrossValidation.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/selection/cv/CrossValidation.java @@ -120,7 +120,7 @@ public CrossValidationResult score(DatasetTrainer trainer, Metric score IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor, int cv, ParamGrid paramGrid) { - List paramSets = new ParameterSetGenerator(paramGrid.getParamValuesByParamIndex()).generate(); + List paramSets = new ParameterSetGenerator(paramGrid.getParamValuesByParamIdx()).generate(); CrossValidationResult cvRes = new CrossValidationResult(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/selection/paramgrid/ParamGrid.java b/modules/ml/src/main/java/org/apache/ignite/ml/selection/paramgrid/ParamGrid.java index 3279d935cf9cd..f9c5bd2b3b9b4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/selection/paramgrid/ParamGrid.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/selection/paramgrid/ParamGrid.java @@ -25,17 +25,17 @@ */ public class ParamGrid { /** Parameter values by parameter index. */ - private Map paramValuesByParamIndex = new HashMap<>(); + private Map paramValuesByParamIdx = new HashMap<>(); /** Parameter names by parameter index. */ - private Map paramNamesByParamIndex = new HashMap<>(); + private Map paramNamesByParamIdx = new HashMap<>(); /** Parameter counter. */ private int paramCntr; /** */ - public Map getParamValuesByParamIndex() { - return paramValuesByParamIndex; + public Map getParamValuesByParamIdx() { + return paramValuesByParamIdx; } /** @@ -45,14 +45,14 @@ public Map getParamValuesByParamIndex() { * @return The updated ParamGrid. */ public ParamGrid addHyperParam(String paramName, Double[] params) { - paramValuesByParamIndex.put(paramCntr, params); - paramNamesByParamIndex.put(paramCntr, paramName); + paramValuesByParamIdx.put(paramCntr, params); + paramNamesByParamIdx.put(paramCntr, paramName); paramCntr++; return this; } /** */ public String getParamNameByIndex(int idx) { - return paramNamesByParamIndex.get(idx); + return paramNamesByParamIdx.get(idx); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java index 573df1a3395df..8fb98d2ae2605 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java @@ -214,7 +214,7 @@ private Deltas calcDeltas(double lb, Vector v, double alpha, double gradient, in double qii = v.dot(v); double newAlpha = calcNewAlpha(alpha, gradient, qii); - Vector deltaWeights = v.times(lb * (newAlpha - alpha) / (this.lambda() * amountOfObservation)); + Vector deltaWeights = v.times(lb * (newAlpha - alpha) / (this.getLambda() * amountOfObservation)); return new Deltas(newAlpha - alpha, deltaWeights); } @@ -233,7 +233,7 @@ private double calcNewAlpha(double alpha, double gradient, double qii) { /** */ private double calcGradient(double lb, Vector v, Vector weights, int amountOfObservation) { double dotProduct = v.dot(weights); - return (lb * dotProduct - 1.0) * (this.lambda() * amountOfObservation); + return (lb * dotProduct - 1.0) * (this.getLambda() * amountOfObservation); } /** */ @@ -261,18 +261,18 @@ public SVMLinearBinaryClassificationTrainer withLambda(double lambda) { } /** - * Gets the regularization lambda. + * Get the regularization lambda. * - * @return The parameter value. + * @return The property value. */ - public double lambda() { + public double getLambda() { return lambda; } /** - * Gets the amount of outer iterations of SCDA algorithm. + * Get the amount of outer iterations of SCDA algorithm. * - * @return The parameter value. + * @return The property value. */ public int getAmountOfIterations() { return amountOfIterations; @@ -290,9 +290,9 @@ public SVMLinearBinaryClassificationTrainer withAmountOfIterations(int amountOfI } /** - * Gets the amount of local iterations of SCDA algorithm. + * Get the amount of local iterations of SCDA algorithm. * - * @return The parameter value. + * @return The property value. */ public int getAmountOfLocIterations() { return amountOfLocIterations; @@ -310,9 +310,9 @@ public SVMLinearBinaryClassificationTrainer withAmountOfLocIterations(int amount } /** - * Gets the seed number. + * Get the seed number. * - * @return The parameter value. + * @return The property value. */ public long getSeed() { return seed; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java index b77baa2a1f84e..aeee1782156e5 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java @@ -82,9 +82,9 @@ public class SVMLinearMultiClassClassificationTrainer classes.forEach(clsLb -> { SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer() - .withAmountOfIterations(this.amountOfIterations()) - .withAmountOfLocIterations(this.amountOfLocIterations()) - .withLambda(this.lambda()) + .withAmountOfIterations(this.getAmountOfIterations()) + .withAmountOfLocIterations(this.getAmountOfLocIterations()) + .withLambda(this.getLambda()) .withSeed(this.seed); IgniteBiFunction lbTransformer = (k, v) -> { @@ -197,20 +197,20 @@ public SVMLinearMultiClassClassificationTrainer withLambda(double lambda) { } /** - * Gets the regularization lambda. + * Get the regularization lambda. * - * @return The parameter value. + * @return The property value. */ - public double lambda() { + public double getLambda() { return lambda; } /** * Gets the amount of outer iterations of SCDA algorithm. * - * @return The parameter value. + * @return The property value. */ - public int amountOfIterations() { + public int getAmountOfIterations() { return amountOfIterations; } @@ -228,9 +228,9 @@ public SVMLinearMultiClassClassificationTrainer withAmountOfIterations(int amoun /** * Gets the amount of local iterations of SCDA algorithm. * - * @return The parameter value. + * @return The property value. */ - public int amountOfLocIterations() { + public int getAmountOfLocIterations() { return amountOfLocIterations; } @@ -248,7 +248,7 @@ public SVMLinearMultiClassClassificationTrainer withAmountOfLocIterations(int am /** * Gets the seed number. * - * @return The parameter value. + * @return The property value. */ public long getSeed() { return seed; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java index 355048a1e4937..45774cb1d7b2d 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java @@ -54,7 +54,7 @@ public abstract class DecisionTree> extends Dataset private final DecisionTreeLeafBuilder decisionTreeLeafBuilder; /** Use index structure instead of using sorting while learning. */ - protected boolean useIndex = true; + protected boolean usingIdx = true; /** * Constructs a new distributed decision tree trainer. @@ -77,7 +77,7 @@ public abstract class DecisionTree> extends Dataset IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { try (Dataset dataset = datasetBuilder.build( new EmptyContextBuilder<>(), - new DecisionTreeDataBuilder<>(featureExtractor, lbExtractor, useIndex) + new DecisionTreeDataBuilder<>(featureExtractor, lbExtractor, usingIdx) )) { return fit(dataset); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainer.java index f8fc769b31d87..91ec8e1498aa3 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainer.java @@ -91,7 +91,7 @@ public DecisionTreeClassificationTrainer withMinImpurityDecrease(Double minImpur * @return Decision tree trainer. */ public DecisionTreeClassificationTrainer withUseIndex(boolean useIndex) { - this.useIndex = useIndex; + this.usingIdx = useIndex; return this; } @@ -127,6 +127,6 @@ else if (b == null) for (Double lb : labels) encoder.put(lb, idx++); - return new GiniImpurityMeasureCalculator(encoder, useIndex); + return new GiniImpurityMeasureCalculator(encoder, usingIdx); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainer.java index 4c9aac9492bcb..ea57bcc6f3967 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainer.java @@ -53,13 +53,13 @@ public DecisionTreeRegressionTrainer(int maxDeep, double minImpurityDecrease, } /** - * Sets useIndex parameter and returns trainer instance. + * Sets usingIdx parameter and returns trainer instance. * - * @param useIndex Use index. + * @param usingIdx Use index. * @return Decision tree trainer. */ - public DecisionTreeRegressionTrainer withUseIndex(boolean useIndex) { - this.useIndex = useIndex; + public DecisionTreeRegressionTrainer withUsingIdx(boolean usingIdx) { + this.usingIdx = usingIdx; return this; } @@ -67,6 +67,6 @@ public DecisionTreeRegressionTrainer withUseIndex(boolean useIndex) { @Override protected ImpurityMeasureCalculator getImpurityMeasureCalculator( Dataset dataset) { - return new MSEImpurityMeasureCalculator(useIndex); + return new MSEImpurityMeasureCalculator(usingIdx); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBBinaryClassifierOnTreesTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBBinaryClassifierOnTreesTrainer.java index 4d87b470b1bb5..b99dc2f5358c5 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBBinaryClassifierOnTreesTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBBinaryClassifierOnTreesTrainer.java @@ -27,13 +27,13 @@ */ public class GDBBinaryClassifierOnTreesTrainer extends GDBBinaryClassifierTrainer { /** Max depth. */ - private final int maxDepth; + private int maxDepth; /** Min impurity decrease. */ - private final double minImpurityDecrease; + private double minImpurityDecrease; - /** Use index structure instead of using sorting while learning. */ - private boolean useIndex = true; + /** Use index structure instead of using sorting during the learning process. */ + private boolean usingIdx = true; /** * Constructs instance of GDBBinaryClassifierOnTreesTrainer. @@ -53,22 +53,71 @@ public GDBBinaryClassifierOnTreesTrainer(double gradStepSize, Integer cntOfItera /** {@inheritDoc} */ @NotNull @Override protected DecisionTreeRegressionTrainer buildBaseModelTrainer() { - return new DecisionTreeRegressionTrainer(maxDepth, minImpurityDecrease).withUseIndex(useIndex); + return new DecisionTreeRegressionTrainer(maxDepth, minImpurityDecrease).withUsingIdx(usingIdx); + } + + /** {@inheritDoc} */ + @Override protected GDBLearningStrategy getLearningStrategy() { + return new GDBOnTreesLearningStrategy(usingIdx); } /** - * Sets useIndex parameter and returns trainer instance. + * Set useIndex parameter and returns trainer instance. * - * @param useIndex Use index. + * @param usingIdx Use index. * @return Decision tree trainer. */ - public GDBBinaryClassifierOnTreesTrainer withUseIndex(boolean useIndex) { - this.useIndex = useIndex; + public GDBBinaryClassifierOnTreesTrainer withUsingIdx(boolean usingIdx) { + this.usingIdx = usingIdx; return this; } - /** {@inheritDoc} */ - @Override protected GDBLearningStrategy getLearningStrategy() { - return new GDBOnTreesLearningStrategy(useIndex); + /** + * Get the max depth. + * + * @return The property value. + */ + public int getMaxDepth() { + return maxDepth; + } + + /** + * Set up the max depth. + * + * @param maxDepth The parameter value. + * @return Decision tree trainer. + */ + public GDBBinaryClassifierOnTreesTrainer setMaxDepth(int maxDepth) { + this.maxDepth = maxDepth; + return this; + } + + /** + * Get the min impurity decrease. + * + * @return The property value. + */ + public double getMinImpurityDecrease() { + return minImpurityDecrease; + } + + /** + * Set up the min impurity decrease. + * + * @param minImpurityDecrease The parameter value. + * @return Decision tree trainer. + */ + public GDBBinaryClassifierOnTreesTrainer setMinImpurityDecrease(double minImpurityDecrease) { + this.minImpurityDecrease = minImpurityDecrease; + return this; + } + + /** + * Get the using index structure property instead of using sorting during the learning process. + * + * @return The property value. + */ + public boolean isUsingIdx() { + return usingIdx; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBRegressionOnTreesTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBRegressionOnTreesTrainer.java index e2a183cd5a3ca..b6c0b486d18da 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBRegressionOnTreesTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/boosting/GDBRegressionOnTreesTrainer.java @@ -27,13 +27,13 @@ */ public class GDBRegressionOnTreesTrainer extends GDBRegressionTrainer { /** Max depth. */ - private final int maxDepth; + private int maxDepth; /** Min impurity decrease. */ - private final double minImpurityDecrease; + private double minImpurityDecrease; /** Use index structure instead of using sorting while learning. */ - private boolean useIndex = true; + private boolean usingIdx = true; /** * Constructs instance of GDBRegressionOnTreesTrainer. @@ -53,22 +53,71 @@ public GDBRegressionOnTreesTrainer(double gradStepSize, Integer cntOfIterations, /** {@inheritDoc} */ @NotNull @Override protected DecisionTreeRegressionTrainer buildBaseModelTrainer() { - return new DecisionTreeRegressionTrainer(maxDepth, minImpurityDecrease).withUseIndex(useIndex); + return new DecisionTreeRegressionTrainer(maxDepth, minImpurityDecrease).withUsingIdx(usingIdx); } /** - * Sets useIndex parameter and returns trainer instance. + * Set useIndex parameter and returns trainer instance. * - * @param useIndex Use index. + * @param usingIdx Use index. * @return Decision tree trainer. */ - public GDBRegressionOnTreesTrainer withUseIndex(boolean useIndex) { - this.useIndex = useIndex; + public GDBRegressionOnTreesTrainer withUsingIdx(boolean usingIdx) { + this.usingIdx = usingIdx; return this; } + /** + * Get the max depth. + * + * @return The property value. + */ + public int getMaxDepth() { + return maxDepth; + } + + /** + * Set up the max depth. + * + * @param maxDepth The parameter value. + * @return Decision tree trainer. + */ + public GDBRegressionOnTreesTrainer setMaxDepth(int maxDepth) { + this.maxDepth = maxDepth; + return this; + } + + /** + * Get the min impurity decrease. + * + * @return The property value. + */ + public double getMinImpurityDecrease() { + return minImpurityDecrease; + } + + /** + * Set up the min impurity decrease. + * + * @param minImpurityDecrease The parameter value. + * @return Decision tree trainer. + */ + public GDBRegressionOnTreesTrainer setMinImpurityDecrease(double minImpurityDecrease) { + this.minImpurityDecrease = minImpurityDecrease; + return this; + } + + /** + * Get the using index structure property instead of using sorting during the learning process. + * + * @return The property value. + */ + public boolean isUsingIdx() { + return usingIdx; + } + /** {@inheritDoc} */ @Override protected GDBLearningStrategy getLearningStrategy() { - return new GDBOnTreesLearningStrategy(useIndex); + return new GDBOnTreesLearningStrategy(usingIdx); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java index 91fcf0ab303ca..c617d8d417303 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/randomforest/RandomForestTrainer.java @@ -76,7 +76,7 @@ public abstract class RandomForestTrainer meta; /** Features per tree. */ - private int featuresPerTree; + private int featuresPerTree = 5; /** Seed. */ - private long seed = System.currentTimeMillis(); + private long seed = 1234L; /** Random generator. */ private Random random = new Random(seed); @@ -115,7 +115,7 @@ public RandomForestTrainer(List meta) { List models = null; try (Dataset dataset = datasetBuilder.build( new EmptyContextBuilder<>(), - new BootstrappedDatasetBuilder<>(featureExtractor, lbExtractor, cntOfTrees, subsampleSize))) { + new BootstrappedDatasetBuilder<>(featureExtractor, lbExtractor, cntOfTrees, subSampleSize))) { if(!init(dataset)) return buildComposition(Collections.emptyList()); @@ -144,11 +144,11 @@ public T withCountOfTrees(int cntOfTrees) { } /** - * @param subsampleSize Subsample size. + * @param subSampleSize Subsample size. * @return an instance of current object with valid type in according to inheritance. */ - public T withSubsampleSize(double subsampleSize) { - this.subsampleSize = subsampleSize; + public T withSubSampleSize(double subSampleSize) { + this.subSampleSize = subSampleSize; return instance(); } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansModelTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansModelTest.java index 03e0e6df5aad9..f71b7b34b69b2 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansModelTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansModelTest.java @@ -54,7 +54,7 @@ public void predictClusters() { Assert.assertEquals(mdl.apply(new DenseVector(new double[]{-1.1, -1.1})), 3.0, PRECISION); Assert.assertEquals(mdl.distanceMeasure(), distanceMeasure); - Assert.assertEquals(mdl.amountOfClusters(), 4); - Assert.assertArrayEquals(mdl.centers(), centers); + Assert.assertEquals(mdl.getAmountOfClusters(), 4); + Assert.assertArrayEquals(mdl.getCenters(), centers); } } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java index 03f044a23a59f..74ff8f19ac1ff 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java @@ -58,7 +58,7 @@ public class KMeansTrainerTest { @Test public void findOneClusters() { KMeansTrainer trainer = createAndCheckTrainer(); - KMeansModel knnMdl = trainer.withK(1).fit( + KMeansModel knnMdl = trainer.withAmountOfClusters(1).fit( new LocalDatasetBuilder<>(data, 2), (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), (k, v) -> v[2] @@ -76,7 +76,7 @@ public void findOneClusters() { @Test public void testUpdateMdl() { KMeansTrainer trainer = createAndCheckTrainer(); - KMeansModel originalMdl = trainer.withK(1).fit( + KMeansModel originalMdl = trainer.withAmountOfClusters(1).fit( new LocalDatasetBuilder<>(data, 2), (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 0, v.length - 1)), (k, v) -> v[2] @@ -106,11 +106,11 @@ public void testUpdateMdl() { @NotNull private KMeansTrainer createAndCheckTrainer() { KMeansTrainer trainer = new KMeansTrainer() .withDistance(new EuclideanDistance()) - .withK(10) + .withAmountOfClusters(10) .withMaxIterations(1) .withEpsilon(PRECISION) .withSeed(2); - assertEquals(10, trainer.getK()); + assertEquals(10, trainer.getAmountOfClusters()); assertEquals(2, trainer.getSeed()); assertTrue(trainer.getDistance() instanceof EuclideanDistance); return trainer; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java index 9315850ace7bf..ca3f0b5549390 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/common/LocalModelsTest.java @@ -196,7 +196,7 @@ private KMeansModel getClusterModel() { data.put(1, new double[] {1.0, 1960, 373200}); KMeansTrainer trainer = new KMeansTrainer() - .withK(1); + .withAmountOfClusters(1); return trainer.fit( new LocalDatasetBuilder<>(data, 2), diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java index 89b8c9c67d5eb..4c3655b3e9e35 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/composition/boosting/GDBTrainerTest.java @@ -57,7 +57,7 @@ public void testFitRegression() { } GDBTrainer trainer = new GDBRegressionOnTreesTrainer(1.0, 2000, 3, 0.0) - .withUseIndex(true); + .withUsingIdx(true); Model mdl = trainer.fit( learningSample, 1, @@ -131,7 +131,7 @@ private void testClassifier(BiFunction, learningSample.put(i, new double[] {xs[i], ys[i]}); GDBTrainer trainer = new GDBBinaryClassifierOnTreesTrainer(0.3, 500, 3, 0.0) - .withUseIndex(true) + .withUsingIdx(true) .withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.3)); Model mdl = fitter.apply(trainer, learningSample); @@ -177,10 +177,10 @@ public void testUpdate() { IgniteBiFunction lExtr = (k, v) -> v[1]; GDBTrainer classifTrainer = new GDBBinaryClassifierOnTreesTrainer(0.3, 500, 3, 0.0) - .withUseIndex(true) + .withUsingIdx(true) .withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.3)); GDBTrainer regressTrainer = new GDBRegressionOnTreesTrainer(0.3, 500, 3, 0.0) - .withUseIndex(true) + .withUsingIdx(true) .withCheckConvergenceStgyFactory(new MeanAbsValueConvergenceCheckerFactory(0.3)); testUpdate(learningSample, fExtr, lExtr, classifTrainer); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java index b06fd6745b6bf..7e5a07977888a 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/environment/LearningEnvironmentTest.java @@ -45,7 +45,7 @@ public void testBasic() throws InterruptedException { .withFeaturesCountSelectionStrgy(FeaturesCountSelectionStrategies.ONE_THIRD) .withMaxDepth(4) .withMinImpurityDelta(0.) - .withSubsampleSize(0.3) + .withSubSampleSize(0.3) .withSeed(0); LearningEnvironment environment = LearningEnvironment.builder() diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainerTest.java index 857d9bdd32d66..d465e82a55cb0 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/binarization/BinarizationTrainerTest.java @@ -69,14 +69,14 @@ public void testFit() { BinarizationTrainer binarizationTrainer = new BinarizationTrainer() .withThreshold(10); - assertEquals(10., binarizationTrainer.threshold(), 0); + assertEquals(10., binarizationTrainer.getThreshold(), 0); BinarizationPreprocessor preprocessor = binarizationTrainer.fit( datasetBuilder, (k, v) -> VectorUtils.of(v) ); - assertEquals(binarizationTrainer.threshold(), preprocessor.threshold(), 0); + assertEquals(binarizationTrainer.getThreshold(), preprocessor.getThreshold(), 0); assertArrayEquals(new double[] {0, 0, 1}, preprocessor.apply(5, new double[] {1, 10, 100}).asArray(), 1e-8); } @@ -93,7 +93,7 @@ public void testFitDefault() { BinarizationTrainer binarizationTrainer = new BinarizationTrainer() .withThreshold(10); - assertEquals(10., binarizationTrainer.threshold(), 0); + assertEquals(10., binarizationTrainer.getThreshold(), 0); IgniteBiFunction preprocessor = binarizationTrainer.fit( data, diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java index f08501ca468a3..73c88428a4ced 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/logistic/LogRegMultiClassTrainerTest.java @@ -61,11 +61,11 @@ public void testTrainWithTheLinearlySeparableCase() { .withBatchSize(100) .withSeed(123L); - Assert.assertEquals(trainer.amountOfIterations(), 1000); - Assert.assertEquals(trainer.amountOfLocIterations(), 10); - Assert.assertEquals(trainer.batchSize(), 100, PRECISION); + Assert.assertEquals(trainer.getAmountOfIterations(), 1000); + Assert.assertEquals(trainer.getAmountOfLocIterations(), 10); + Assert.assertEquals(trainer.getBatchSize(), 100, PRECISION); Assert.assertEquals(trainer.seed(), 123L); - Assert.assertEquals(trainer.updatesStgy(), stgy); + Assert.assertEquals(trainer.getUpdatesStgy(), stgy); LogRegressionMultiClassModel mdl = trainer.fit( cacheMock, diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java index 4e649257f6edc..84975a86d1e38 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java @@ -73,7 +73,7 @@ public void testFit() { } DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(1, 0) - .withUseIndex(useIndex == 1); + .withUsingIdx(useIndex == 1); DecisionTreeNode tree = trainer.fit( data, diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.java index ed474fe9429fe..9fa7f0ec590bf 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/randomforest/RandomForestTest.java @@ -60,7 +60,7 @@ public class RandomForestTest { .withFeaturesCountSelectionStrgy(x -> 4) .withMaxDepth(maxDepth) .withMinImpurityDelta(minImpDelta) - .withSubsampleSize(0.1); + .withSubSampleSize(0.1); /** */ @Test From ec48acdedf6dfd4c1e8daeb2c2a254d920613989 Mon Sep 17 00:00:00 2001 From: Aleksey Plekhanov Date: Fri, 7 Sep 2018 14:22:48 +0300 Subject: [PATCH 64/95] IGNITE-9366: SQL: Added IGNITE.NODE_METRICS system view. This closes #4615. --- .../processors/query/h2/IgniteH2Indexing.java | 2 + .../sys/view/SqlAbstractLocalSystemView.java | 27 +++ .../h2/sys/view/SqlSystemViewNodeMetrics.java | 210 ++++++++++++++++ .../query/SqlSystemViewsSelfTest.java | 226 ++++++++++++++++-- 4 files changed, 448 insertions(+), 17 deletions(-) create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java 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 cc2363b07e0b5..f94430a6f3ade 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 @@ -130,6 +130,7 @@ import org.apache.ignite.internal.processors.query.h2.sys.SqlSystemTableEngine; import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewBaselineNodes; import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeAttributes; +import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeMetrics; import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodes; import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor; import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor; @@ -3208,6 +3209,7 @@ public Collection systemViews(GridKernalContext ctx) { views.add(new SqlSystemViewNodes(ctx)); views.add(new SqlSystemViewNodeAttributes(ctx)); views.add(new SqlSystemViewBaselineNodes(ctx)); + views.add(new SqlSystemViewNodeMetrics(ctx)); return views; } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java index ac90b631b8297..d692dbac3dafa 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java @@ -26,6 +26,8 @@ import org.h2.value.Value; import org.h2.value.ValueNull; import org.h2.value.ValueString; +import org.h2.value.ValueTime; +import org.h2.value.ValueTimestamp; /** * Local system view base class (which uses only local node data). @@ -125,4 +127,29 @@ protected static UUID uuidFromValue(Value val) { return null; } } + + /** + * Converts millis to ValueTime + * + * @param millis Millis. + */ + protected static Value valueTimeFromMillis(long millis) { + if (millis == -1L || millis == Long.MAX_VALUE) + return ValueNull.INSTANCE; + else + // Note: ValueTime.fromMillis(long) method trying to convert time using timezone and return wrong result. + return ValueTime.fromNanos(millis * 1_000_000L); + } + + /** + * Converts millis to ValueTimestamp + * + * @param millis Millis. + */ + protected static Value valueTimestampFromMillis(long millis) { + if (millis <= 0L || millis == Long.MAX_VALUE) + return ValueNull.INSTANCE; + else + return ValueTimestamp.fromMillis(millis); + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java new file mode 100644 index 0000000000000..01b4e976f0cae --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java @@ -0,0 +1,210 @@ +/* + * 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.sys.view; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; +import org.apache.ignite.cluster.ClusterMetrics; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.util.typedef.F; +import org.h2.engine.Session; +import org.h2.result.Row; +import org.h2.result.SearchRow; +import org.h2.value.Value; + +/** + * System view: node metrics. + */ +public class SqlSystemViewNodeMetrics extends SqlAbstractLocalSystemView { + /** + * @param ctx Grid context. + */ + public SqlSystemViewNodeMetrics(GridKernalContext ctx) { + super("NODE_METRICS", "Node metrics", ctx, new String[] {"NODE_ID"}, + newColumn("NODE_ID", Value.UUID), + newColumn("LAST_UPDATE_TIME", Value.TIMESTAMP), + newColumn("MAX_ACTIVE_JOBS", Value.INT), + newColumn("CUR_ACTIVE_JOBS", Value.INT), + newColumn("AVG_ACTIVE_JOBS", Value.FLOAT), + newColumn("MAX_WAITING_JOBS", Value.INT), + newColumn("CUR_WAITING_JOBS", Value.INT), + newColumn("AVG_WAITING_JOBS", Value.FLOAT), + newColumn("MAX_REJECTED_JOBS", Value.INT), + newColumn("CUR_REJECTED_JOBS", Value.INT), + newColumn("AVG_REJECTED_JOBS", Value.FLOAT), + newColumn("TOTAL_REJECTED_JOBS", Value.INT), + newColumn("MAX_CANCELED_JOBS", Value.INT), + newColumn("CUR_CANCELED_JOBS", Value.INT), + newColumn("AVG_CANCELED_JOBS", Value.FLOAT), + newColumn("TOTAL_CANCELED_JOBS", Value.INT), + newColumn("MAX_JOBS_WAIT_TIME", Value.TIME), + newColumn("CUR_JOBS_WAIT_TIME", Value.TIME), + newColumn("AVG_JOBS_WAIT_TIME", Value.TIME), + newColumn("MAX_JOBS_EXECUTE_TIME", Value.TIME), + newColumn("CUR_JOBS_EXECUTE_TIME", Value.TIME), + newColumn("AVG_JOBS_EXECUTE_TIME", Value.TIME), + newColumn("TOTAL_JOBS_EXECUTE_TIME", Value.TIME), + newColumn("TOTAL_EXECUTED_JOBS", Value.INT), + newColumn("TOTAL_EXECUTED_TASKS", Value.INT), + newColumn("TOTAL_BUSY_TIME", Value.TIME), + newColumn("TOTAL_IDLE_TIME", Value.TIME), + newColumn("CUR_IDLE_TIME", Value.TIME), + newColumn("BUSY_TIME_PERCENTAGE", Value.FLOAT), + newColumn("IDLE_TIME_PERCENTAGE", Value.FLOAT), + newColumn("TOTAL_CPU", Value.INT), + newColumn("CUR_CPU_LOAD", Value.DOUBLE), + newColumn("AVG_CPU_LOAD", Value.DOUBLE), + newColumn("CUR_GC_CPU_LOAD", Value.DOUBLE), + newColumn("HEAP_MEMORY_INIT", Value.LONG), + newColumn("HEAP_MEMORY_USED", Value.LONG), + newColumn("HEAP_MEMORY_COMMITED", Value.LONG), + newColumn("HEAP_MEMORY_MAX", Value.LONG), + newColumn("HEAP_MEMORY_TOTAL", Value.LONG), + newColumn("NONHEAP_MEMORY_INIT", Value.LONG), + newColumn("NONHEAP_MEMORY_USED", Value.LONG), + newColumn("NONHEAP_MEMORY_COMMITED", Value.LONG), + newColumn("NONHEAP_MEMORY_MAX", Value.LONG), + newColumn("NONHEAP_MEMORY_TOTAL", Value.LONG), + newColumn("UPTIME", Value.TIME), + newColumn("JVM_START_TIME", Value.TIMESTAMP), + newColumn("NODE_START_TIME", Value.TIMESTAMP), + newColumn("LAST_DATA_VERSION", Value.LONG), + newColumn("CUR_THREAD_COUNT", Value.INT), + newColumn("MAX_THREAD_COUNT", Value.INT), + newColumn("TOTAL_THREAD_COUNT", Value.LONG), + newColumn("CUR_DAEMON_THREAD_COUNT", Value.INT), + newColumn("SENT_MESSAGES_COUNT", Value.INT), + newColumn("SENT_BYTES_COUNT", Value.LONG), + newColumn("RECEIVED_MESSAGES_COUNT", Value.INT), + newColumn("RECEIVED_BYTES_COUNT", Value.LONG), + newColumn("OUTBOUND_MESSAGES_QUEUE", Value.INT) + ); + } + + /** {@inheritDoc} */ + @Override public Iterator getRows(Session ses, SearchRow first, SearchRow last) { + List rows = new ArrayList<>(); + + Collection nodes; + + SqlSystemViewColumnCondition idCond = conditionForColumn("NODE_ID", first, last); + + if (idCond.isEquality()) { + try { + UUID nodeId = uuidFromValue(idCond.valueForEquality()); + + ClusterNode node = nodeId == null ? null : ctx.discovery().node(nodeId); + + if (node != null) + nodes = Collections.singleton(node); + else + nodes = Collections.emptySet(); + } + catch (Exception e) { + nodes = Collections.emptySet(); + } + } + else + nodes = F.concat(false, ctx.discovery().allNodes(), ctx.discovery().daemonNodes()); + + for (ClusterNode node : nodes) { + if (node != null) { + ClusterMetrics metrics = node.metrics(); + + rows.add( + createRow(ses, rows.size(), + node.id(), + valueTimestampFromMillis(metrics.getLastUpdateTime()), + metrics.getMaximumActiveJobs(), + metrics.getCurrentActiveJobs(), + metrics.getAverageActiveJobs(), + metrics.getMaximumWaitingJobs(), + metrics.getCurrentWaitingJobs(), + metrics.getAverageWaitingJobs(), + metrics.getMaximumRejectedJobs(), + metrics.getCurrentRejectedJobs(), + metrics.getAverageRejectedJobs(), + metrics.getTotalRejectedJobs(), + metrics.getMaximumCancelledJobs(), + metrics.getCurrentCancelledJobs(), + metrics.getAverageCancelledJobs(), + metrics.getTotalCancelledJobs(), + valueTimeFromMillis(metrics.getMaximumJobWaitTime()), + valueTimeFromMillis(metrics.getCurrentJobWaitTime()), + valueTimeFromMillis((long)metrics.getAverageJobWaitTime()), + valueTimeFromMillis(metrics.getMaximumJobExecuteTime()), + valueTimeFromMillis(metrics.getCurrentJobExecuteTime()), + valueTimeFromMillis((long)metrics.getAverageJobExecuteTime()), + valueTimeFromMillis(metrics.getTotalJobsExecutionTime()), + metrics.getTotalExecutedJobs(), + metrics.getTotalExecutedTasks(), + valueTimeFromMillis(metrics.getTotalBusyTime()), + valueTimeFromMillis(metrics.getTotalIdleTime()), + valueTimeFromMillis(metrics.getCurrentIdleTime()), + metrics.getBusyTimePercentage(), + metrics.getIdleTimePercentage(), + metrics.getTotalCpus(), + metrics.getCurrentCpuLoad(), + metrics.getAverageCpuLoad(), + metrics.getCurrentGcCpuLoad(), + metrics.getHeapMemoryInitialized(), + metrics.getHeapMemoryUsed(), + metrics.getHeapMemoryCommitted(), + metrics.getHeapMemoryMaximum(), + metrics.getHeapMemoryTotal(), + metrics.getNonHeapMemoryInitialized(), + metrics.getNonHeapMemoryUsed(), + metrics.getNonHeapMemoryCommitted(), + metrics.getNonHeapMemoryMaximum(), + metrics.getNonHeapMemoryTotal(), + valueTimeFromMillis(metrics.getUpTime()), + valueTimestampFromMillis(metrics.getStartTime()), + valueTimestampFromMillis(metrics.getNodeStartTime()), + metrics.getLastDataVersion(), + metrics.getCurrentThreadCount(), + metrics.getMaximumThreadCount(), + metrics.getTotalStartedThreadCount(), + metrics.getCurrentDaemonThreadCount(), + metrics.getSentMessagesCount(), + metrics.getSentBytesCount(), + metrics.getReceivedMessagesCount(), + metrics.getReceivedBytesCount(), + metrics.getOutboundMessagesQueueSize() + ) + ); + } + } + + return rows.iterator(); + } + + /** {@inheritDoc} */ + @Override public boolean canGetRowCount() { + return true; + } + + /** {@inheritDoc} */ + @Override public long getRowCount() { + return F.concat(false, ctx.discovery().allNodes(), ctx.discovery().daemonNodes()).size(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java index c8374a78ffc2f..1a4dae794fb10 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java @@ -17,21 +17,31 @@ package org.apache.ignite.internal.processors.query; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Collections; import java.util.List; +import java.util.Random; +import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.Callable; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.ClusterMetricsSnapshot; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -39,6 +49,9 @@ * Tests for ignite SQL system views. */ public class SqlSystemViewsSelfTest extends GridCommonAbstractTest { + /** Metrics check attempts. */ + private static final int METRICS_CHECK_ATTEMPTS = 10; + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); @@ -171,7 +184,7 @@ public void testCacheToViewJoin() throws Exception { private void assertColumnTypes(List rowData, Class ... colTypes) { for (int i = 0; i < colTypes.length; i++) { if (rowData.get(i) != null) - assertEquals("Column " + i + " type", rowData.get(i).getClass(), colTypes[i]); + assertEquals("Column " + i + " type", colTypes[i], rowData.get(i).getClass()); } } @@ -181,9 +194,14 @@ private void assertColumnTypes(List rowData, Class ... colTypes) { * @throws Exception If failed. */ public void testNodesViews() throws Exception { - Ignite ignite1 = startGrid(getTestIgniteInstanceName(), getConfiguration()); - Ignite ignite2 = startGrid(getTestIgniteInstanceName(1), getConfiguration().setClientMode(true)); - Ignite ignite3 = startGrid(getTestIgniteInstanceName(2), getConfiguration().setDaemon(true)); + Ignite igniteSrv = startGrid(getTestIgniteInstanceName(), getConfiguration().setMetricsUpdateFrequency(500L)); + + Ignite igniteCli = startGrid(getTestIgniteInstanceName(1), getConfiguration().setMetricsUpdateFrequency(500L) + .setClientMode(true)); + + startGrid(getTestIgniteInstanceName(2), getConfiguration().setMetricsUpdateFrequency(500L).setDaemon(true)); + + UUID nodeId0 = igniteSrv.cluster().localNode().id(); awaitPartitionMapExchange(); @@ -201,7 +219,7 @@ public void testNodesViews() throws Exception { assertEquals(1, resSrv.size()); - assertEquals(ignite1.cluster().localNode().id(), resSrv.get(0).get(0)); + assertEquals(nodeId0, resSrv.get(0).get(0)); assertEquals(1, resSrv.get(0).get(1)); @@ -210,7 +228,7 @@ public void testNodesViews() throws Exception { assertEquals(1, resCli.size()); - assertEquals(ignite2.cluster().localNode().id(), resCli.get(0).get(0)); + assertEquals(nodeId(1), resCli.get(0).get(0)); assertEquals(2, resCli.get(0).get(1)); @@ -219,7 +237,7 @@ public void testNodesViews() throws Exception { assertEquals(1, resDaemon.size()); - assertEquals(ignite3.cluster().localNode().id(), resDaemon.get(0).get(0)); + assertEquals(nodeId(2), resDaemon.get(0).get(0)); assertEquals(3, resDaemon.get(0).get(1)); @@ -227,32 +245,30 @@ public void testNodesViews() throws Exception { assertEquals(0, execSql("SELECT ID FROM IGNITE.NODES WHERE ID = '-'").size()); assertEquals(1, execSql("SELECT ID FROM IGNITE.NODES WHERE ID = ?", - ignite1.cluster().localNode().id()).size()); + nodeId0).size()); assertEquals(1, execSql("SELECT ID FROM IGNITE.NODES WHERE ID = ?", - ignite3.cluster().localNode().id()).size()); + nodeId(2)).size()); // Check index on ID column with disjunction. assertEquals(3, execSql("SELECT ID FROM IGNITE.NODES WHERE ID = ? " + - "OR node_order=1 OR node_order=2 OR node_order=3", ignite1.cluster().localNode().id()).size()); + "OR node_order=1 OR node_order=2 OR node_order=3", nodeId0).size()); // Check quick-count. assertEquals(3L, execSql("SELECT COUNT(*) FROM IGNITE.NODES").get(0).get(0)); // Check joins - assertEquals(ignite1.cluster().localNode().id(), execSql("SELECT N1.ID FROM IGNITE.NODES N1 JOIN " + + assertEquals(nodeId0, execSql("SELECT N1.ID FROM IGNITE.NODES N1 JOIN " + "IGNITE.NODES N2 ON N1.NODE_ORDER = N2.NODE_ORDER JOIN IGNITE.NODES N3 ON N2.ID = N3.ID " + "WHERE N3.NODE_ORDER = 1") .get(0).get(0)); // Check sub-query - assertEquals(ignite1.cluster().localNode().id(), execSql("SELECT N1.ID FROM IGNITE.NODES N1 " + + assertEquals(nodeId0, execSql("SELECT N1.ID FROM IGNITE.NODES N1 " + "WHERE NOT EXISTS (SELECT 1 FROM IGNITE.NODES N2 WHERE N2.ID = N1.ID AND N2.NODE_ORDER <> 1)") .get(0).get(0)); // Check node attributes view - UUID cliNodeId = ignite2.cluster().localNode().id(); - String cliAttrName = IgniteNodeAttributes.ATTR_CLIENT_MODE; assertColumnTypes(execSql("SELECT NODE_ID, NAME, VALUE FROM IGNITE.NODE_ATTRIBUTES").get(0), @@ -267,7 +283,7 @@ public void testNodesViews() throws Exception { assertEquals(1, execSql("SELECT NODE_ID FROM IGNITE.NODE_ATTRIBUTES WHERE NODE_ID = ? AND NAME = ? AND VALUE = 'true'", - cliNodeId, cliAttrName).size()); + nodeId(1), cliAttrName).size()); assertEquals(0, execSql("SELECT NODE_ID FROM IGNITE.NODE_ATTRIBUTES WHERE NODE_ID = '-' AND NAME = ?", @@ -275,7 +291,169 @@ public void testNodesViews() throws Exception { assertEquals(0, execSql("SELECT NODE_ID FROM IGNITE.NODE_ATTRIBUTES WHERE NODE_ID = ? AND NAME = '-'", - cliNodeId).size()); + nodeId(1)).size()); + + // Check node metrics view. + String sqlAllMetrics = "SELECT NODE_ID, LAST_UPDATE_TIME, " + + "MAX_ACTIVE_JOBS, CUR_ACTIVE_JOBS, AVG_ACTIVE_JOBS, " + + "MAX_WAITING_JOBS, CUR_WAITING_JOBS, AVG_WAITING_JOBS, " + + "MAX_REJECTED_JOBS, CUR_REJECTED_JOBS, AVG_REJECTED_JOBS, TOTAL_REJECTED_JOBS, " + + "MAX_CANCELED_JOBS, CUR_CANCELED_JOBS, AVG_CANCELED_JOBS, TOTAL_CANCELED_JOBS, " + + "MAX_JOBS_WAIT_TIME, CUR_JOBS_WAIT_TIME, AVG_JOBS_WAIT_TIME, " + + "MAX_JOBS_EXECUTE_TIME, CUR_JOBS_EXECUTE_TIME, AVG_JOBS_EXECUTE_TIME, TOTAL_JOBS_EXECUTE_TIME, " + + "TOTAL_EXECUTED_JOBS, TOTAL_EXECUTED_TASKS, " + + "TOTAL_BUSY_TIME, TOTAL_IDLE_TIME, CUR_IDLE_TIME, BUSY_TIME_PERCENTAGE, IDLE_TIME_PERCENTAGE, " + + "TOTAL_CPU, CUR_CPU_LOAD, AVG_CPU_LOAD, CUR_GC_CPU_LOAD, " + + "HEAP_MEMORY_INIT, HEAP_MEMORY_USED, HEAP_MEMORY_COMMITED, HEAP_MEMORY_MAX, HEAP_MEMORY_TOTAL, " + + "NONHEAP_MEMORY_INIT, NONHEAP_MEMORY_USED, NONHEAP_MEMORY_COMMITED, NONHEAP_MEMORY_MAX, NONHEAP_MEMORY_TOTAL, " + + "UPTIME, JVM_START_TIME, NODE_START_TIME, LAST_DATA_VERSION, " + + "CUR_THREAD_COUNT, MAX_THREAD_COUNT, TOTAL_THREAD_COUNT, CUR_DAEMON_THREAD_COUNT, " + + "SENT_MESSAGES_COUNT, SENT_BYTES_COUNT, RECEIVED_MESSAGES_COUNT, RECEIVED_BYTES_COUNT, " + + "OUTBOUND_MESSAGES_QUEUE FROM IGNITE.NODE_METRICS"; + + List> resMetrics = execSql(sqlAllMetrics); + + assertColumnTypes(resMetrics.get(0), UUID.class, Timestamp.class, + Integer.class, Integer.class, Float.class, // Active jobs. + Integer.class, Integer.class, Float.class, // Waiting jobs. + Integer.class, Integer.class, Float.class, Integer.class, // Rejected jobs. + Integer.class, Integer.class, Float.class, Integer.class, // Canceled jobs. + Time.class, Time.class, Time.class, // Jobs wait time. + Time.class, Time.class, Time.class, Time.class, // Jobs execute time. + Integer.class, Integer.class, // Executed jobs/task. + Time.class, Time.class, Time.class, Float.class, Float.class, // Busy/idle time. + Integer.class, Double.class, Double.class, Double.class, // CPU. + Long.class, Long.class, Long.class, Long.class, Long.class, // Heap memory. + Long.class, Long.class, Long.class, Long.class, Long.class, // Nonheap memory. + Time.class, Timestamp.class, Timestamp.class, Long.class, // Uptime. + Integer.class, Integer.class, Long.class, Integer.class, // Threads. + Integer.class, Long.class, Integer.class, Long.class, // Sent/received messages. + Integer.class); // Outbound message queue. + + assertEquals(3, resAll.size()); + + // Check join with nodes. + assertEquals(3, execSql("SELECT NM.LAST_UPDATE_TIME FROM IGNITE.NODES N " + + "JOIN IGNITE.NODE_METRICS NM ON N.ID = NM.NODE_ID").size()); + + // Check index on NODE_ID column. + assertEquals(1, execSql("SELECT LAST_UPDATE_TIME FROM IGNITE.NODE_METRICS WHERE NODE_ID = ?", + nodeId(1)).size()); + + // Check malformed value for indexed column. + assertEquals(0, execSql("SELECT LAST_UPDATE_TIME FROM IGNITE.NODE_METRICS WHERE NODE_ID = ?", + "-").size()); + + // Check quick-count. + assertEquals(3L, execSql("SELECT COUNT(*) FROM IGNITE.NODE_METRICS").get(0).get(0)); + + // Check metric values. + + // Broadcast jobs to server and client nodes to get non zero metric values. + for (int i = 0; i < 100; i++) { + IgniteFuture fut = igniteSrv.compute(igniteSrv.cluster().forNodeId(nodeId0, nodeId(1))) + .broadcastAsync( + new IgniteRunnable() { + @Override public void run() { + Random rnd = new Random(); + + try { + doSleep(rnd.nextInt(100)); + } + catch (Throwable ignore) { + // No-op. + } + } + }); + + if (i % 10 == 0) + fut.cancel(); + } + + doSleep(igniteSrv.configuration().getMetricsUpdateFrequency() * 3L); + + for (Ignite grid : G.allGrids()) { + UUID nodeId = grid.cluster().localNode().id(); + + // Metrics for node must be collected from another node to avoid race and get consistent metrics snapshot. + Ignite ignite = F.eq(nodeId, nodeId0) ? igniteCli : igniteSrv; + + for (int i = 0; i < METRICS_CHECK_ATTEMPTS; i++) { + ClusterMetrics metrics = ignite.cluster().node(nodeId).metrics(); + + assertTrue(metrics instanceof ClusterMetricsSnapshot); + + resMetrics = execSql(ignite, sqlAllMetrics + " WHERE NODE_ID = ?", nodeId); + + log.info("Check metrics for node " + grid.name() + ", attempt " + (i + 1)); + + if (metrics.getLastUpdateTime() == ((Timestamp)resMetrics.get(0).get(1)).getTime()) { + assertEquals(metrics.getMaximumActiveJobs(), resMetrics.get(0).get(2)); + assertEquals(metrics.getCurrentActiveJobs(), resMetrics.get(0).get(3)); + assertEquals(metrics.getAverageActiveJobs(), resMetrics.get(0).get(4)); + assertEquals(metrics.getMaximumWaitingJobs(), resMetrics.get(0).get(5)); + assertEquals(metrics.getCurrentWaitingJobs(), resMetrics.get(0).get(6)); + assertEquals(metrics.getAverageWaitingJobs(), resMetrics.get(0).get(7)); + assertEquals(metrics.getMaximumRejectedJobs(), resMetrics.get(0).get(8)); + assertEquals(metrics.getCurrentRejectedJobs(), resMetrics.get(0).get(9)); + assertEquals(metrics.getAverageRejectedJobs(), resMetrics.get(0).get(10)); + assertEquals(metrics.getTotalRejectedJobs(), resMetrics.get(0).get(11)); + assertEquals(metrics.getMaximumCancelledJobs(), resMetrics.get(0).get(12)); + assertEquals(metrics.getCurrentCancelledJobs(), resMetrics.get(0).get(13)); + assertEquals(metrics.getAverageCancelledJobs(), resMetrics.get(0).get(14)); + assertEquals(metrics.getTotalCancelledJobs(), resMetrics.get(0).get(15)); + assertEquals(metrics.getMaximumJobWaitTime(), convertToMilliseconds(resMetrics.get(0).get(16))); + assertEquals(metrics.getCurrentJobWaitTime(), convertToMilliseconds(resMetrics.get(0).get(17))); + assertEquals((long)metrics.getAverageJobWaitTime(), convertToMilliseconds(resMetrics.get(0).get(18))); + assertEquals(metrics.getMaximumJobExecuteTime(), convertToMilliseconds(resMetrics.get(0).get(19))); + assertEquals(metrics.getCurrentJobExecuteTime(), convertToMilliseconds(resMetrics.get(0).get(20))); + assertEquals((long)metrics.getAverageJobExecuteTime(), convertToMilliseconds(resMetrics.get(0).get(21))); + assertEquals(metrics.getTotalJobsExecutionTime(), convertToMilliseconds(resMetrics.get(0).get(22))); + assertEquals(metrics.getTotalExecutedJobs(), resMetrics.get(0).get(23)); + assertEquals(metrics.getTotalExecutedTasks(), resMetrics.get(0).get(24)); + assertEquals(metrics.getTotalBusyTime(), convertToMilliseconds(resMetrics.get(0).get(25))); + assertEquals(metrics.getTotalIdleTime(), convertToMilliseconds(resMetrics.get(0).get(26))); + assertEquals(metrics.getCurrentIdleTime(), convertToMilliseconds(resMetrics.get(0).get(27))); + assertEquals(metrics.getBusyTimePercentage(), resMetrics.get(0).get(28)); + assertEquals(metrics.getIdleTimePercentage(), resMetrics.get(0).get(29)); + assertEquals(metrics.getTotalCpus(), resMetrics.get(0).get(30)); + assertEquals(metrics.getCurrentCpuLoad(), resMetrics.get(0).get(31)); + assertEquals(metrics.getAverageCpuLoad(), resMetrics.get(0).get(32)); + assertEquals(metrics.getCurrentGcCpuLoad(), resMetrics.get(0).get(33)); + assertEquals(metrics.getHeapMemoryInitialized(), resMetrics.get(0).get(34)); + assertEquals(metrics.getHeapMemoryUsed(), resMetrics.get(0).get(35)); + assertEquals(metrics.getHeapMemoryCommitted(), resMetrics.get(0).get(36)); + assertEquals(metrics.getHeapMemoryMaximum(), resMetrics.get(0).get(37)); + assertEquals(metrics.getHeapMemoryTotal(), resMetrics.get(0).get(38)); + assertEquals(metrics.getNonHeapMemoryInitialized(), resMetrics.get(0).get(39)); + assertEquals(metrics.getNonHeapMemoryUsed(), resMetrics.get(0).get(40)); + assertEquals(metrics.getNonHeapMemoryCommitted(), resMetrics.get(0).get(41)); + assertEquals(metrics.getNonHeapMemoryMaximum(), resMetrics.get(0).get(42)); + assertEquals(metrics.getNonHeapMemoryTotal(), resMetrics.get(0).get(43)); + assertEquals(metrics.getUpTime(), convertToMilliseconds(resMetrics.get(0).get(44))); + assertEquals(metrics.getStartTime(), ((Timestamp)resMetrics.get(0).get(45)).getTime()); + assertEquals(metrics.getNodeStartTime(), ((Timestamp)resMetrics.get(0).get(46)).getTime()); + assertEquals(metrics.getLastDataVersion(), resMetrics.get(0).get(47)); + assertEquals(metrics.getCurrentThreadCount(), resMetrics.get(0).get(48)); + assertEquals(metrics.getMaximumThreadCount(), resMetrics.get(0).get(49)); + assertEquals(metrics.getTotalStartedThreadCount(), resMetrics.get(0).get(50)); + assertEquals(metrics.getCurrentDaemonThreadCount(), resMetrics.get(0).get(51)); + assertEquals(metrics.getSentMessagesCount(), resMetrics.get(0).get(52)); + assertEquals(metrics.getSentBytesCount(), resMetrics.get(0).get(53)); + assertEquals(metrics.getReceivedMessagesCount(), resMetrics.get(0).get(54)); + assertEquals(metrics.getReceivedBytesCount(), resMetrics.get(0).get(55)); + assertEquals(metrics.getOutboundMessagesQueueSize(), resMetrics.get(0).get(56)); + + break; + } + else { + log.info("Metrics was updated in background, will retry check"); + + if (i == METRICS_CHECK_ATTEMPTS - 1) + fail("Failed to check metrics, attempts limit reached (" + METRICS_CHECK_ATTEMPTS + ')'); + } + } + } } /** @@ -322,7 +500,7 @@ public void testBaselineViews() throws Exception { } /** - * Gets ignite configuration with persistance enabled. + * Gets ignite configuration with persistence enabled. */ private IgniteConfiguration getPdsConfiguration(String consistentId) throws Exception { IgniteConfiguration cfg = getConfiguration(); @@ -336,4 +514,18 @@ private IgniteConfiguration getPdsConfiguration(String consistentId) throws Exce return cfg; } + + /** + * Convert Time to milliseconds. + * + * Note: Returned Time values from SQL it's milliseconds since January 1, 1970, 00:00:00 GMT. To get right interval + * in milliseconds this value must be adjusted to current time zone. + * + * @param sqlTime Time value returned from SQL. + */ + private long convertToMilliseconds(Object sqlTime) { + Time time0 = (Time)sqlTime; + + return time0.getTime() + TimeZone.getDefault().getOffset(time0.getTime()); + } } From 831d2a35c597e5a211c670e6242254c18440119a Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 7 Sep 2018 15:16:02 +0300 Subject: [PATCH 65/95] IGNITE-9161: Optimization for C++ (copy elision) This closes #4476 --- .../include/ignite/binary/binary_raw_reader.h | 22 ++- .../ignite/impl/binary/binary_reader_impl.h | 11 ++ .../ignite/impl/binary/binary_type_impl.h | 16 +- .../cpp/core/include/ignite/cache/cache.h | 46 ++++-- .../include/ignite/cache/query/query_cursor.h | 15 +- .../cpp/core/include/ignite/impl/operations.h | 150 ++++++------------ .../cpp/odbc-test/src/connection_test.cpp | 7 +- 7 files changed, 128 insertions(+), 139 deletions(-) diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h index c06cb91edf848..73d25256721f0 100644 --- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h +++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h @@ -312,19 +312,31 @@ namespace ignite * @return String. */ std::string ReadString() + { + std::string res; + + ReadString(res); + + return res; + } + + /** + * Read string from the stream. + * + * @param dst String. + */ + void ReadString(std::string& dst) { int32_t len = ReadString(NULL, 0); if (len != -1) { - ignite::common::FixedSizeArray arr(len + 1); - - ReadString(arr.GetData(), static_cast(arr.GetSize())); + dst.resize(static_cast(len)); - return std::string(arr.GetData()); + ReadString(&dst[0], len); } else - return std::string(); + dst.clear(); } /** diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h index 94f5ec517d325..a3b880c01b416 100644 --- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h +++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h @@ -899,6 +899,17 @@ namespace ignite return ignite::binary::ReadHelper::Read(*this); } + /** + * Read object. + * + * @param res Read object. + */ + template + void ReadTopObject(T& res) + { + return ignite::binary::ReadHelper::Read(*this, res); + } + /** * Read object. * diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h index 64638c2bb5d48..e9bbace63611f 100644 --- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h +++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h @@ -97,10 +97,16 @@ namespace ignite { T res; - reader.template ReadTopObject0(res); + Read(reader, res); return res; } + + template + static void Read(R& reader, T& val) + { + reader.template ReadTopObject0(val); + } }; /** @@ -117,10 +123,16 @@ namespace ignite std::auto_ptr res(new T()); - reader.template ReadTopObject0(*res); + reader.template ReadTopObject0(*res); return res.release(); } + + template + static void Read(R& reader, T*& ptr) + { + ptr = Read(reader); + } }; } } diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache.h b/modules/platforms/cpp/core/include/ignite/cache/cache.h index b16d7f5af1dca..c23036195c0a3 100644 --- a/modules/platforms/cpp/core/include/ignite/cache/cache.h +++ b/modules/platforms/cpp/core/include/ignite/cache/cache.h @@ -259,12 +259,14 @@ namespace ignite */ V LocalPeek(const K& key, int32_t peekModes, IgniteError& err) { + V val; + impl::InCacheLocalPeekOperation inOp(key, peekModes); - impl::Out1Operation outOp; + impl::Out1Operation outOp(val); impl.Get()->LocalPeek(inOp, outOp, peekModes, err); - return outOp.GetResult(); + return val; } /** @@ -305,12 +307,13 @@ namespace ignite */ V Get(const K& key, IgniteError& err) { + V val; impl::In1Operation inOp(key); - impl::Out1Operation outOp; + impl::Out1Operation outOp(val); impl.Get()->Get(inOp, outOp, err); - return outOp.GetResult(); + return val; } /** @@ -351,12 +354,14 @@ namespace ignite */ std::map GetAll(const std::set& keys, IgniteError& err) { + std::map res; + impl::InSetOperation inOp(keys); - impl::OutMapOperation outOp; + impl::OutMapOperation outOp(res); impl.Get()->GetAll(inOp, outOp, err); - return outOp.GetResult(); + return res; } /** @@ -515,12 +520,14 @@ namespace ignite */ V GetAndPut(const K& key, const V& val, IgniteError& err) { + V oldVal; + impl::In2Operation inOp(key, val); - impl::Out1Operation outOp; + impl::Out1Operation outOp(oldVal); impl.Get()->GetAndPut(inOp, outOp, err); - return outOp.GetResult(); + return oldVal; } /** @@ -559,12 +566,14 @@ namespace ignite */ V GetAndReplace(const K& key, const V& val, IgniteError& err) { + V oldVal; + impl::In2Operation inOp(key, val); - impl::Out1Operation outOp; + impl::Out1Operation outOp(oldVal); impl.Get()->GetAndReplace(inOp, outOp, err); - return outOp.GetResult(); + return oldVal; } /** @@ -597,12 +606,14 @@ namespace ignite */ V GetAndRemove(const K& key, IgniteError& err) { + V oldVal; + impl::In1Operation inOp(key); - impl::Out1Operation outOp; + impl::Out1Operation outOp(oldVal); impl.Get()->GetAndRemove(inOp, outOp, err); - return outOp.GetResult(); + return oldVal; } /** @@ -694,12 +705,14 @@ namespace ignite */ V GetAndPutIfAbsent(const K& key, const V& val, IgniteError& err) { + V oldVal; + impl::In2Operation inOp(key, val); - impl::Out1Operation outOp; + impl::Out1Operation outOp(oldVal); impl.Get()->GetAndPutIfAbsent(inOp, outOp, err); - return outOp.GetResult(); + return oldVal; } /** @@ -1607,14 +1620,15 @@ namespace ignite { typedef impl::cache::CacheEntryProcessorHolder ProcessorHolder; + R res; ProcessorHolder procHolder(processor, arg); impl::In2Operation inOp(key, procHolder); - impl::Out1Operation outOp; + impl::Out1Operation outOp(res); impl.Get()->Invoke(inOp, outOp, err); - return outOp.GetResult(); + return res; } /** diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h index 3f7ccce3e8e68..e77706b1c772b 100644 --- a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h +++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h @@ -158,19 +158,14 @@ namespace ignite impl::cache::query::QueryCursorImpl* impl0 = impl.Get(); if (impl0) { - impl::Out2Operation outOp; + K key; + V val; - impl0->GetNext(outOp, err); + impl::Out2Operation outOp(key, val); - if (err.GetCode() == IgniteError::IGNITE_SUCCESS) - { - K& key = outOp.Get1(); - V& val = outOp.Get2(); + impl0->GetNext(outOp, err); - return CacheEntry(key, val); - } - else - return CacheEntry(); + return CacheEntry(key, val); } else { diff --git a/modules/platforms/cpp/core/include/ignite/impl/operations.h b/modules/platforms/cpp/core/include/ignite/impl/operations.h index 1645fb5691633..9f816bf7250aa 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/operations.h +++ b/modules/platforms/cpp/core/include/ignite/impl/operations.h @@ -66,7 +66,7 @@ namespace ignite public: /** * Constructor. - * + * * @param val Value. */ In1Operation(const T& val) : val(val) @@ -248,7 +248,7 @@ namespace ignite const T& key; /** Peek modes. */ - int32_t peekModes; + int32_t peekModes; IGNITE_NO_COPY_ASSIGNMENT(InCacheLocalPeekOperation) }; @@ -333,15 +333,18 @@ namespace ignite public: /** * Constructor. + * + * @param val Value. */ - Out1Operation() + Out1Operation(T& val) : + val(val) { // No-op. } virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { - val = reader.ReadTopObject(); + reader.ReadTopObject(val); } virtual void SetNull() @@ -349,18 +352,9 @@ namespace ignite val = binary::BinaryUtils::GetDefaultValue(); } - /** - * Get value. - * - * @param Value. - */ - T GetResult() - { - return val; - } private: /** Value. */ - T val; + T& val; IGNITE_NO_COPY_ASSIGNMENT(Out1Operation) }; @@ -374,16 +368,21 @@ namespace ignite public: /** * Constructor. + * + * @param val1 Value 1. + * @param val2 Value 2. */ - Out2Operation() + Out2Operation(T1& val1, T2& val2) : + val1(val1), + val2(val2) { // No-op. } virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { - val1 = reader.ReadTopObject(); - val2 = reader.ReadTopObject(); + reader.ReadTopObject(val1); + reader.ReadTopObject(val2); } virtual void SetNull() @@ -392,32 +391,12 @@ namespace ignite val2 = binary::BinaryUtils::GetDefaultValue(); } - /** - * Get value 1. - * - * @param Value 1. - */ - T1& Get1() - { - return val1; - } - - /** - * Get value 2. - * - * @param Value 2. - */ - T2& Get2() - { - return val2; - } - private: /** Value 1. */ - T1 val1; - + T1& val1; + /** Value 2. */ - T2 val2; + T2& val2; IGNITE_NO_COPY_ASSIGNMENT(Out2Operation) }; @@ -431,18 +410,27 @@ namespace ignite public: /** * Constructor. + * + * @param val1 Value 1. + * @param val2 Value 2. + * @param val3 Value 3. + * @param val4 Value 4. */ - Out4Operation() + Out4Operation(T1& val1, T2& val2, T3& val3, T4& val4) : + val1(val1), + val2(val2), + val3(val3), + val4(val4) { // No-op. } virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { - val1 = reader.ReadTopObject(); - val2 = reader.ReadTopObject(); - val3 = reader.ReadTopObject(); - val4 = reader.ReadTopObject(); + reader.ReadTopObject(val1); + reader.ReadTopObject(val2); + reader.ReadTopObject(val3); + reader.ReadTopObject(val4); } virtual void SetNull() @@ -453,58 +441,18 @@ namespace ignite val4 = binary::BinaryUtils::GetDefaultValue(); } - /** - * Get value 1. - * - * @param Value 1. - */ - T1& Get1() - { - return val1; - } - - /** - * Get value 2. - * - * @param Value 2. - */ - T2& Get2() - { - return val2; - } - - /** - * Get value 3. - * - * @param Value 3. - */ - T3& Get3() - { - return val3; - } - - /** - * Get value 4. - * - * @param Value 4. - */ - T4& Get4() - { - return val4; - } - private: /** Value 1. */ - T1 val1; - + T1& val1; + /** Value 2. */ - T2 val2; + T2& val2; /** Value 3. */ - T3 val3; + T3& val3; /** Value 4. */ - T4 val4; + T4& val4; IGNITE_NO_COPY_ASSIGNMENT(Out4Operation) }; @@ -518,8 +466,11 @@ namespace ignite public: /** * Constructor. + * + * @param val Value. */ - OutMapOperation() + OutMapOperation(std::map& val) : + val(val) { // No-op. } @@ -550,18 +501,9 @@ namespace ignite // No-op. } - /** - * Get value. - * - * @return Value. - */ - std::map GetResult() - { - return val; - } private: /** Value. */ - std::map val; + std::map& val; IGNITE_NO_COPY_ASSIGNMENT(OutMapOperation) }; @@ -587,7 +529,7 @@ namespace ignite res.reserve(res.size() + cnt); - for (int i = 0; i < cnt; i++) + for (int i = 0; i < cnt; i++) { K key = reader.ReadTopObject(); V val = reader.ReadTopObject(); @@ -604,7 +546,7 @@ namespace ignite private: /** Entries. */ std::vector >& res; - + IGNITE_NO_COPY_ASSIGNMENT(OutQueryGetAllOperation) }; @@ -645,7 +587,7 @@ namespace ignite private: /** Out iter. */ Iter iter; - + IGNITE_NO_COPY_ASSIGNMENT(OutQueryGetAllOperationIter) }; diff --git a/modules/platforms/cpp/odbc-test/src/connection_test.cpp b/modules/platforms/cpp/odbc-test/src/connection_test.cpp index 5badda65a55d5..709ef61857c37 100644 --- a/modules/platforms/cpp/odbc-test/src/connection_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/connection_test.cpp @@ -84,12 +84,15 @@ struct ConnectionTestSuiteFixture: odbc::OdbcTestSuite /** * Extract code from ODBC error message. + * + * @param err Error. + * @return Code. */ - std::string ExtractErrorCode(std::string err) + static std::string ExtractErrorCode(const std::string& err) { std::string code; - int idx = err.find(':'); + size_t idx = err.find(':'); if ((idx != std::string::npos) && (idx > 0)) code = err.substr(0, idx); From a6e5e74c18d14e918db1dc9a47c282564114569d Mon Sep 17 00:00:00 2001 From: ipavlukhin Date: Fri, 7 Sep 2018 16:00:02 +0300 Subject: [PATCH 66/95] IGNITE-8149: MVCC: correct processing of cache size. This closes #4654. --- .../communication/GridIoMessageFactory.java | 4 +- .../processors/cache/GridCacheMapEntry.java | 12 + .../cache/IgniteCacheOffheapManager.java | 8 + .../cache/IgniteCacheOffheapManagerImpl.java | 54 +- .../GridDistributedTxRemoteAdapter.java | 51 +- .../dht/GridDhtTxAbstractEnlistFuture.java | 2 +- .../dht/GridDhtTxFinishFuture.java | 6 +- .../dht/GridDhtTxFinishRequest.java | 14 +- .../dht/GridDhtTxLocalAdapter.java | 51 +- .../distributed/dht/GridDhtTxRemote.java | 12 - ...sMap.java => PartitionUpdateCounters.java} | 22 +- .../persistence/GridCacheOffheapManager.java | 13 + .../cache/transactions/IgniteInternalTx.java | 8 +- .../cache/transactions/IgniteTxAdapter.java | 57 +- .../cache/transactions/IgniteTxHandler.java | 2 +- .../transactions/IgniteTxLocalAdapter.java | 104 ++-- .../cache/transactions/IgniteTxLocalEx.java | 8 + .../transactions/IgniteTxLocalState.java | 16 + .../IgniteTxLocalStateAdapter.java | 26 + .../cache/transactions/IgniteTxRemoteEx.java | 12 - .../cache/transactions/TxCounters.java | 82 +++ .../cache/mvcc/CacheMvccSizeTest.java | 488 ++++++++++++++++++ .../IgniteCacheMvccSqlTestSuite.java | 5 +- 23 files changed, 880 insertions(+), 177 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/{GridDhtPartitionsUpdateCountersMap.java => PartitionUpdateCounters.java} (86%) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index 2970e71c41339..14f0548a866d3 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 @@ -70,7 +70,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest; @@ -80,6 +79,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryEnlistResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxQueryFirstEnlistRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCounters; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest; @@ -1062,7 +1062,7 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; case 157: - msg = new GridDhtPartitionsUpdateCountersMap(); + msg = new PartitionUpdateCounters(); break; 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 8c2b939bb5b5d..714d4a0e0719c 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 @@ -1122,6 +1122,9 @@ else if (op == CREATE && tx.local() && (res.resultType() == ResultType.PREV_NOT_ if (tx.local()) updateCntr = nextMvccPartitionCounter(); + if (res.resultType() == ResultType.PREV_NULL) + tx.txCounters(true).accumulateSizeDelta(cctx.cacheId(), partition(), 1); + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) logPtr = cctx.shared().wal().log(new DataRecord(new DataEntry( cctx.cacheId(), @@ -1217,6 +1220,9 @@ else if (res.resultType() == ResultType.LOCKED) { if (tx.local()) updateCntr = nextMvccPartitionCounter(); + if (res.resultType() == ResultType.PREV_NOT_NULL) + tx.txCounters(true).accumulateSizeDelta(cctx.cacheId(), partition(), -1); + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) logPtr = logTxUpdate(tx, null, 0, updateCntr); @@ -4994,6 +5000,9 @@ else if (res.resultType() == ResultType.LOCKED) { if (updateCntr != null && updateCntr != 0) updateCntr0 = updateCntr; + if (res.resultType() == ResultType.PREV_NOT_NULL) + tx.txCounters(true).accumulateSizeDelta(cctx.cacheId(), entry.partition(), -1); + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) logPtr = cctx.shared().wal().log(new DataRecord(new DataEntry( cctx.cacheId(), @@ -5281,6 +5290,9 @@ else if (op == CREATE && tx.local() && (res.resultType() == ResultType.PREV_NOT_ updateCntr0 = tx.local() ? entry.nextMvccPartitionCounter() : updateCntr; + if (res.resultType() == ResultType.PREV_NULL) + tx.txCounters(true).accumulateSizeDelta(cctx.cacheId(), entry.partition(), 1); + if (cctx.group().persistenceEnabled() && cctx.group().walEnabled()) logPtr = cctx.shared().wal().log(new DataRecord(new DataEntry( cctx.cacheId(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java index a0213940de37e..fdf42febde2a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java @@ -619,6 +619,14 @@ interface CacheDataStore { */ long fullSize(); + /** + * Updates size metric for particular cache. + * + * @param cacheId Cache ID. + * @param delta Size delta. + */ + void updateSize(int cacheId, long delta); + /** * @return Update counter. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 13ad7e2d5236c..8811006505e26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -1450,36 +1450,14 @@ public CacheDataStoreImpl( * @param cacheId Cache ID. */ void incrementSize(int cacheId) { - storageSize.incrementAndGet(); - - if (grp.sharedGroup()) { - AtomicLong size = cacheSizes.get(cacheId); - - if (size == null) { - AtomicLong old = cacheSizes.putIfAbsent(cacheId, size = new AtomicLong()); - - if (old != null) - size = old; - } - - size.incrementAndGet(); - } + updateSize(cacheId, 1); } /** * @param cacheId Cache ID. */ void decrementSize(int cacheId) { - storageSize.decrementAndGet(); - - if (grp.sharedGroup()) { - AtomicLong size = cacheSizes.get(cacheId); - - if (size == null) - return; - - size.decrementAndGet(); - } + updateSize(cacheId, -1); } /** {@inheritDoc} */ @@ -1516,6 +1494,24 @@ void decrementSize(int cacheId) { return storageSize.get(); } + /** {@inheritDoc} */ + @Override public void updateSize(int cacheId, long delta) { + storageSize.addAndGet(delta); + + if (grp.sharedGroup()) { + AtomicLong size = cacheSizes.get(cacheId); + + if (size == null) { + AtomicLong old = cacheSizes.putIfAbsent(cacheId, size = new AtomicLong()); + + if (old != null) + size = old; + } + + size.addAndGet(delta); + } + } + /** {@inheritDoc} */ @Override public long nextUpdateCounter() { return cntr.incrementAndGet(); @@ -1953,8 +1949,6 @@ else if (res == ResultType.PREV_NULL && noCreate) { assert !old; - incrementSize(cctx.cacheId()); - GridCacheQueryManager qryMgr = cctx.queries(); if (qryMgr.enabled()) @@ -2284,11 +2278,13 @@ else if (res == ResultType.PREV_NOT_NULL) { rowStore.removeRow(row.link()); - decrementSize(cctx.cacheId()); - if (first) first = false; } + + // first == true means there were no row versions + if (!first) + decrementSize(cctx.cacheId()); } /** {@inheritDoc} */ @@ -2318,8 +2314,6 @@ else if (res == ResultType.PREV_NOT_NULL) { rowStore.removeRow(cleanupRow.link()); - decrementSize(cctx.cacheId()); - res++; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index dd6ea48e1b33c..8e96ae2dcee54 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -51,7 +51,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; +import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCounters; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; @@ -61,6 +62,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteEx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteState; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxState; +import org.apache.ignite.internal.processors.cache.transactions.TxCounters; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; @@ -807,7 +809,7 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in } } - updateLocalCounters(); + applyTxCounters(); if (!near() && !F.isEmpty(dataEntries) && cctx.wal() != null) { // Set new update counters for data entries received from persisted tx entries. @@ -845,29 +847,34 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in } } - /** - * Applies update counters to the local partitions. - */ - private void updateLocalCounters() { - Map updCntrsMap = updateCountersMap(); + /** {@inheritDoc} */ + @Override protected void applyTxCounters() { + super.applyTxCounters(); + + TxCounters txCntrs = txCounters(false); - if (F.isEmpty(updCntrsMap)) + if (txCntrs == null) return; - for (Map.Entry entry : updCntrsMap.entrySet()) { - GridCacheContext cacheCtx = cctx.cacheContext(entry.getKey()); + Map updCntrs = txCntrs.updateCounters(); - GridDhtPartitionsUpdateCountersMap cacheUpdCntrs = entry.getValue(); + for (Map.Entry entry : updCntrs.entrySet()) { + int cacheId = entry.getKey(); - assert cacheUpdCntrs != null && !F.isEmpty(cacheUpdCntrs.updateCounters()); + GridDhtPartitionTopology top = cctx.cacheContext(cacheId).topology(); - for (Map.Entry e : cacheUpdCntrs.updateCounters().entrySet()) { - Long updCntr = e.getValue(); - GridDhtLocalPartition part = cacheCtx.topology().localPartition(e.getKey()); + Map cacheUpdCntrs = entry.getValue().updateCounters(); - assert part != null && updCntr != null && updCntr > 0; + assert cacheUpdCntrs != null; - part.updateCounter(updCntr); + for (Map.Entry e : cacheUpdCntrs.entrySet()) { + long updCntr = e.getValue(); + + GridDhtLocalPartition dhtPart = top.localPartition(e.getKey()); + + assert dhtPart != null && updCntr > 0; + + dhtPart.updateCounter(updCntr); } } } @@ -1022,16 +1029,6 @@ protected void addExplicit(IgniteTxEntry e) { } } - /** {@inheritDoc} */ - @Override public void updateCountersMap(Map updCntrsMap) { - // No-op. - } - - /** {@inheritDoc} */ - @Override public Map updateCountersMap() { - return null; - } - /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(GridDistributedTxRemoteAdapter.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java index bb11df5590c62..a3471c7e3d13a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java @@ -593,7 +593,7 @@ private void addToBatch(KeyCacheObject key, CacheObject val, List updCntrsMap = null; + Map updCntrsForNode = null; if (dhtMapping.queryUpdate() && commit) - updCntrsMap = tx.updateCountersForNode(n); + updCntrsForNode = tx.filterUpdateCountersForBackupNode(n); GridDhtTxFinishRequest req = new GridDhtTxFinishRequest( tx.nearNodeId(), @@ -489,7 +489,7 @@ private boolean finish(boolean commit, false, false, mvccSnapshot, - updCntrsMap); + updCntrsForNode); req.writeVersion(tx.writeVersion() != null ? tx.writeVersion() : tx.xidVersion()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java index b1908ff4120ad..8e9ece6cb3c03 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java @@ -73,9 +73,10 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest { /** */ private MvccSnapshot mvccSnapshot; - /** Map of update counters made by this tx. Mapping: cacheId -> partId -> updCntr. */ - @GridDirectMap(keyType = Integer.class, valueType = GridDhtPartitionsUpdateCountersMap.class) - private Map updCntrs; + /** */ + @GridDirectMap(keyType = Integer.class, valueType = PartitionUpdateCounters.class) + private Map updCntrs; + /** * Empty constructor required for {@link Externalizable}. */ @@ -220,7 +221,7 @@ public GridDhtTxFinishRequest( boolean retVal, boolean waitRemoteTxs, MvccSnapshot mvccSnapshot, - Map updCntrs + Map updCntrs ) { this(nearNodeId, futId, @@ -362,10 +363,11 @@ public boolean needReturnValue() { public void needReturnValue(boolean retVal) { setFlag(retVal, NEED_RETURN_VALUE_FLAG_MASK); } + /** - * @return Partition update counters map. + * @return Partition counters update deferred until transaction commit. */ - public Map updateCountersMap() { + public Map updateCounters() { return updCntrs; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index 11b46a0f2fab8..613f1600f03d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -20,6 +20,7 @@ import java.io.Externalizable; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -31,6 +32,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; @@ -42,6 +44,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter; +import org.apache.ignite.internal.processors.cache.transactions.TxCounters; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanMap; @@ -907,7 +910,7 @@ public GridFutureAdapter finishFuture(GridFutureAdapter f, Throwable e * * @return Current lock future or null if it's safe to roll back. */ - public @Nullable IgniteInternalFuture tryRollbackAsync() { + @Nullable public IgniteInternalFuture tryRollbackAsync() { while (true) { final IgniteInternalFuture fut = lockFut; @@ -937,6 +940,52 @@ protected final IgniteInternalFuture chainOnePhasePre return prepFut; } + /** + * @param node Backup node. + * @return Partition counters map for the given backup node. + */ + public Map filterUpdateCountersForBackupNode(ClusterNode node) { + TxCounters txCntrs = txCounters(false); + + if (txCntrs == null) + return null; + + Map updCntrs = txCntrs.updateCounters(); + + Map res = new HashMap<>(); + + AffinityTopologyVersion top = topologyVersionSnapshot(); + + for (Map.Entry entry : updCntrs.entrySet()) { + Integer cacheId = entry.getKey(); + + Map partsCntrs = entry.getValue().updateCounters(); + + assert !F.isEmpty(partsCntrs); + + GridCacheAffinityManager affinity = cctx.cacheContext(cacheId).affinity(); + + Map resCntrs = new HashMap<>(partsCntrs.size()); + + for (Map.Entry e : partsCntrs.entrySet()) { + Integer p = e.getKey(); + + Long cntr = e.getValue(); + + if (affinity.backupByPartition(node, p, top)) { + assert cntr != null && cntr > 0 : cntr; + + resCntrs.put(p, cntr); + } + } + + if (!resCntrs.isEmpty()) + res.put(cacheId, new PartitionUpdateCounters(resCntrs)); + } + + return res; + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(GridDhtTxLocalAdapter.class, this, "nearNodes", nearMap.keySet(), 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 08ecf28e3a3b0..9a1763be4051f 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 @@ -79,8 +79,6 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter { /** Store write through flag. */ private boolean storeWriteThrough; - /** Map of update counters made by this tx. Mapping: cacheId -> partId -> updCntr. */ - private Map updCntrs; /** * Empty constructor required for {@link Externalizable}. */ @@ -502,16 +500,6 @@ public void mvccEnlistBatch(GridCacheContext ctx, EnlistOperation op, List updCntrsMap) { - this.updCntrs = updCntrsMap; - } - - /** {@inheritDoc} */ - @Override public Map updateCountersMap() { - return updCntrs; - } - /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(GridDhtTxRemote.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsUpdateCountersMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCounters.java similarity index 86% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsUpdateCountersMap.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCounters.java index 7a345d13dfbba..5b1eccdce033c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsUpdateCountersMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCounters.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.Map; import org.apache.ignite.internal.GridDirectMap; import org.apache.ignite.plugin.extensions.communication.Message; @@ -27,21 +26,26 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter; /** - * Partitions update counters message. + * Partition update counters message. */ -public class GridDhtPartitionsUpdateCountersMap implements Message { +public class PartitionUpdateCounters implements Message { /** */ - private static final long serialVersionUID = -4599730112233297219L; + private static final long serialVersionUID = 193442457510062844L; /** Map of update counters made by this tx. Mapping: partId -> updCntr. */ @GridDirectMap(keyType = Integer.class, valueType = Long.class) - private Map updCntrs; + private Map updCntrs; + + /** */ + public PartitionUpdateCounters() { + // No-op. + } /** - * + * @param updCntrs Update counters map. */ - public GridDhtPartitionsUpdateCountersMap() { - updCntrs = new HashMap<>(); + public PartitionUpdateCounters(Map updCntrs) { + this.updCntrs = updCntrs; } /** @@ -99,7 +103,7 @@ public void updateCounters(Map updCntrs) { } - return reader.afterMessageRead(GridDhtPartitionsUpdateCountersMap.class); + return reader.afterMessageRead(PartitionUpdateCounters.class); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index 199efcb59446d..04476ad0cb814 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -1562,6 +1562,19 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { } } + /** {@inheritDoc} */ + @Override public void updateSize(int cacheId, long delta) { + try { + CacheDataStore delegate0 = init0(false); + + if (delegate0 != null) + delegate0.updateSize(cacheId, delta); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + /** {@inheritDoc} */ @Override public long updateCounter() { try { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java index 4acf078bc8038..05ebe5d31496b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java @@ -650,4 +650,10 @@ public void completedVersions(GridCacheVersion base, * @return Mvcc snapshot. */ public MvccSnapshot mvccSnapshot(); -} \ No newline at end of file + + /** + * @return Transaction counters. + * @param createIfAbsent {@code True} if non-null instance is needed. + */ + @Nullable public TxCounters txCounters(boolean createIfAbsent); +} 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 ee5a58ee49129..c6d19919f868a 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 @@ -34,6 +34,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import javax.cache.expiry.ExpiryPolicy; @@ -62,10 +63,12 @@ import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; import org.apache.ignite.internal.processors.cache.version.GridCacheLazyPlainVersionedEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -136,6 +139,10 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement private static final AtomicReferenceFieldUpdater FINALIZING_UPD = AtomicReferenceFieldUpdater.newUpdater(IgniteTxAdapter.class, FinalizationStatus.class, "finalizing"); + /** */ + private static final AtomicReferenceFieldUpdater TX_COUNTERS_UPD = + AtomicReferenceFieldUpdater.newUpdater(IgniteTxAdapter.class, TxCounters.class, "txCounters"); + /** Logger. */ protected static IgniteLogger log; @@ -275,6 +282,9 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement @GridToStringExclude private volatile IgniteInternalFuture rollbackFut; + /** */ + private volatile TxCounters txCounters = new TxCounters(); + /** * Empty constructor required for {@link Externalizable}. */ @@ -2014,6 +2024,46 @@ protected Object readResolve() throws ObjectStreamException { */ public abstract void addActiveCache(GridCacheContext cacheCtx, boolean recovery) throws IgniteCheckedException; + /** {@inheritDoc} */ + @Nullable @Override public TxCounters txCounters(boolean createIfAbsent) { + if (createIfAbsent && txCounters == null) + TX_COUNTERS_UPD.compareAndSet(this, null, new TxCounters()); + + return txCounters; + } + + /** + * Make counters accumulated during transaction visible outside of transaciton. + */ + protected void applyTxCounters() { + TxCounters txCntrs = txCounters(false); + + if (txCntrs == null) + return; + + Map> sizeDeltas = txCntrs.sizeDeltas(); + + for (Map.Entry> entry : sizeDeltas.entrySet()) { + Integer cacheId = entry.getKey(); + Map partDeltas = entry.getValue(); + + assert !F.isEmpty(partDeltas); + + GridDhtPartitionTopology top = cctx.cacheContext(cacheId).topology(); + + for (Map.Entry e : partDeltas.entrySet()) { + Integer p = e.getKey(); + long delta = e.getValue().get(); + + GridDhtLocalPartition dhtPart = top.localPartition(p); + + assert dhtPart != null; + + dhtPart.dataStore().updateSize(cacheId, delta); + } + } + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(IgniteTxAdapter.class, this, @@ -2286,6 +2336,11 @@ private static class TxShadow implements IgniteInternalTx { throw new IllegalStateException("Deserialized transaction can only be used as read-only."); } + /** {@inheritDoc} */ + @Nullable @Override public TxCounters txCounters(boolean createIfAbsent) { + return null; + } + /** {@inheritDoc} */ @Override public IgniteTxState txState() { return 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 7541b433ec478..32f4dd43352f0 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 @@ -1391,7 +1391,7 @@ else if (log.isDebugEnabled()) tx.setPartitionUpdateCounters( req.partUpdateCounters() != null ? req.partUpdateCounters().array() : null); - tx.updateCountersMap(req.updateCountersMap()); + tx.txCounters(true).updateCounters(req.updateCounters()); tx.commitRemoteTx(); } 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 b86273f9f1a6e..bfe67ee0fdeda 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 @@ -24,7 +24,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import javax.cache.expiry.Duration; @@ -32,7 +31,6 @@ import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.failure.FailureContext; import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.IgniteInternalFuture; @@ -59,7 +57,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCounters; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.persistence.StorageException; @@ -162,9 +160,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig /** */ private GridLongList mvccWaitTxs; - /** Update counters map */ - private Map> updCntrs; - /** */ private volatile boolean qryEnlisted; @@ -918,7 +913,7 @@ else if (!isNodeStopping) { // Skip fair uncommit in case of node stopping or in } } - updateLocalPartitionCounters(); + applyTxCounters(); if (ptr != null && !cctx.tm().logTxRecords()) cctx.wal().flush(ptr, false); @@ -1562,6 +1557,11 @@ public Map> partsMap() { return null; } + /** {@inheritDoc} */ + @Override public void touchPartition(int cacheId, int partId) { + txState.touchPartition(cacheId, partId); + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(IgniteTxLocalAdapter.class, this, "super", super.toString(), @@ -1644,74 +1644,28 @@ public long entryExpireTime(IgniteTxKey key) { } /** - * @return Partition counters map for the given backup node. + * Merges mvcc update counters to the partition update counters. For mvcc transactions we update partitions + * counters only on commit phase. */ - public Map updateCountersForNode(ClusterNode node) { - if (F.isEmpty(updCntrs)) + private Map applyAndCollectLocalUpdateCounters() { + if (F.isEmpty(txState.touchedPartitions())) return null; - Map res = new HashMap<>(); + HashMap updCntrs = new HashMap<>(); - for (Map.Entry> entry : updCntrs.entrySet()) { - Map partsCntrs = entry.getValue(); + for (Map.Entry> entry : txState.touchedPartitions().entrySet()) { + Integer cacheId = entry.getKey(); - assert !F.isEmpty(partsCntrs); + Set parts = entry.getValue(); - GridCacheContext ctx0 = cctx.cacheContext(entry.getKey()); + assert !F.isEmpty(parts); - GridDhtPartitionsUpdateCountersMap resBackupCntrs = new GridDhtPartitionsUpdateCountersMap(); + GridCacheContext ctx0 = cctx.cacheContext(cacheId); - for (Map.Entry e : partsCntrs.entrySet()) { - Long cntr = partsCntrs.get(e.getKey()); + Map partCntrs = new HashMap<>(parts.size()); - if (ctx0.affinity().backupByPartition(node, e.getKey(), topologyVersionSnapshot())) { - assert cntr != null && cntr > 0 : cntr; - - resBackupCntrs.updateCounters().put(e.getKey(), cntr); - } - } - - if (!resBackupCntrs.updateCounters().isEmpty()) - res.put(entry.getKey(), resBackupCntrs); - } - - return res; - } - - /** - * @param cacheId Cache id. - * @param part Partition id. - */ - @SuppressWarnings("Java8MapApi") - public void addPartitionCountersMapping(Integer cacheId, Integer part) { - if (updCntrs == null) - updCntrs = new ConcurrentHashMap<>(); - - Map partUpdCntrs = updCntrs.get(cacheId); - - if (partUpdCntrs == null) - updCntrs.put(cacheId, partUpdCntrs = new ConcurrentHashMap<>()); - - partUpdCntrs.put(part, 0L); - } - - /** - * Merges mvcc update counters to the partition update counters. For mvcc transactions we update partitions - * counters only on commit phase. - */ - private void updateLocalPartitionCounters() { - if (F.isEmpty(updCntrs)) - return; - - for (Map.Entry> entry : updCntrs.entrySet()) { - Map partsCntrs = entry.getValue(); - - assert !F.isEmpty(partsCntrs); - - GridCacheContext ctx0 = cctx.cacheContext(entry.getKey()); - - for (Map.Entry e : partsCntrs.entrySet()) { - GridDhtLocalPartition dhtPart = ctx0.topology().localPartition(e.getKey()); + for (Integer p : parts) { + GridDhtLocalPartition dhtPart = ctx0.topology().localPartition(p); assert dhtPart != null; @@ -1719,11 +1673,13 @@ private void updateLocalPartitionCounters() { dhtPart.updateCounter(cntr); - Long prev = partsCntrs.put(e.getKey(), cntr); - - assert prev == 0L : prev; + partCntrs.put(p, cntr); } + + updCntrs.put(cacheId, new PartitionUpdateCounters(partCntrs)); } + + return updCntrs; } /** @@ -1747,6 +1703,16 @@ public void markQueryEnlisted(MvccSnapshot ver) { } } + /** {@inheritDoc} */ + @Override protected void applyTxCounters() { + super.applyTxCounters(); + + Map updCntrs = applyAndCollectLocalUpdateCounters(); + + // remember counters for subsequent sending to backups + txCounters(true).updateCounters(updCntrs); + } + /** * Post-lock closure alias. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java index b61b1a9a629b5..651be603d37a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java @@ -56,4 +56,12 @@ public interface IgniteTxLocalEx extends IgniteInternalTx { * @throws IgniteCheckedException If finish failed. */ public boolean localFinish(boolean commit, boolean clearThreadMap) throws IgniteCheckedException; + + /** + * Remembers that particular cache partition was touched by current tx. + * + * @param cacheId Cache id. + * @param partId Partition id. + */ + public void touchPartition(int cacheId, int partId); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java index 123d3964705b8..01eb4f4417877 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalState.java @@ -17,6 +17,9 @@ package org.apache.ignite.internal.processors.cache.transactions; +import java.util.Map; +import java.util.Set; + /** * */ @@ -41,4 +44,17 @@ public interface IgniteTxLocalState extends IgniteTxState { * */ public void seal(); + + /** + * @return Cache partitions touched by current tx. + */ + public Map> touchedPartitions(); + + /** + * Remembers that particular cache partition was touched by current tx. + * + * @param cacheId Cache id. + * @param partId Partition id. + */ + public void touchPartition(int cacheId, int partId); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java index 4943aac863b24..9c6ef8f3a91e1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalStateAdapter.java @@ -17,7 +17,13 @@ package org.apache.ignite.internal.processors.cache.transactions; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.util.typedef.internal.U; @@ -25,6 +31,11 @@ * */ public abstract class IgniteTxLocalStateAdapter implements IgniteTxLocalState { + /** */ + private static final Function> CREATE_INT_SET = k -> new HashSet<>(); + /** */ + private Map> touchedParts; + /** * @param cacheCtx Cache context. * @param tx Transaction. @@ -40,4 +51,19 @@ protected final void onTxEnd(GridCacheContext cacheCtx, IgniteInternalTx tx, boo cacheCtx.cache().metrics0().onTxRollback(durationNanos); } } + + /** {@inheritDoc} */ + @Override public Map> touchedPartitions() { + Map> parts = touchedParts; + + return parts != null ? Collections.unmodifiableMap(parts) : null; + } + + /** {@inheritDoc} */ + @Override public void touchPartition(int cacheId, int partId) { + if (touchedParts == null) + touchedParts = new HashMap<>(); + + touchedParts.computeIfAbsent(cacheId, CREATE_INT_SET).add(partId); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java index 1e0645ffb3ab5..87cc7ccea3635 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java @@ -18,9 +18,7 @@ package org.apache.ignite.internal.processors.cache.transactions; import java.util.Collection; -import java.util.Map; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsUpdateCountersMap; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; /** @@ -52,14 +50,4 @@ public void doneRemote(GridCacheVersion baseVer, * @param cntrs Partition update indexes. */ public void setPartitionUpdateCounters(long[] cntrs); - - /** - * @param updCntrsMap Partition update counters map: cacheId -> partId -> updateCntr. - */ - public void updateCountersMap(Map updCntrsMap); - - /** - * @return Partition update counters map: cacheId -> partId -> updateCntr. - */ - public Map updateCountersMap(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java new file mode 100644 index 0000000000000..2ad4f9410d2f0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java @@ -0,0 +1,82 @@ +/* + * 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.Collections; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCounters; + +/** + * Values which should be tracked during transaction execution and applied on commit. + */ +public class TxCounters { + /** Size changes for cache partitions made by transaction */ + private final ConcurrentMap> sizeDeltas = new ConcurrentHashMap<>(); + /** Update counters for cache partitions in the end of transaction */ + private Map updCntrs; + + /** + * Accumulates size change for cache partition. + * + * @param cacheId Cache id. + * @param part Partition id. + * @param delta Size delta. + */ + public void accumulateSizeDelta(int cacheId, int part, long delta) { + ConcurrentMap partDeltas = sizeDeltas.get(cacheId); + + if (partDeltas == null) { + ConcurrentMap partDeltas0 = + sizeDeltas.putIfAbsent(cacheId, partDeltas = new ConcurrentHashMap<>()); + + if (partDeltas0 != null) + partDeltas = partDeltas0; + } + + AtomicLong accDelta = partDeltas.get(part); + + if (accDelta == null) { + AtomicLong accDelta0 = partDeltas.putIfAbsent(part, accDelta = new AtomicLong()); + + if (accDelta0 != null) + accDelta = accDelta0; + } + + // here AtomicLong is used more as a container, + // every instance is assumed to be accessed in thread-confined manner + accDelta.set(accDelta.get() + delta); + } + + /** */ + public void updateCounters(Map updCntrs) { + this.updCntrs = updCntrs; + } + + /** */ + public Map updateCounters() { + return updCntrs != null ? Collections.unmodifiableMap(updCntrs) : Collections.emptyMap(); + } + + /** */ + public Map> sizeDeltas() { + return Collections.unmodifiableMap(sizeDeltas); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeTest.java new file mode 100644 index 0000000000000..32709ffbb496d --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSizeTest.java @@ -0,0 +1,488 @@ +/* + * 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.mvcc; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Consumer; +import java.util.stream.IntStream; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.query.IgniteSQLException; + +import static org.apache.ignite.cache.CachePeekMode.BACKUP; + +/** + * + */ +public class CacheMvccSizeTest extends CacheMvccAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return CacheMode.PARTITIONED; + } + + /** */ + private void checkSizeModificationByOperation(String sql, boolean commit, int expSizeDelta) throws Exception { + checkSizeModificationByOperation(c -> {}, cache -> cache.query(q(sql)).getAll(), commit, expSizeDelta); + } + + /** */ + private void checkSizeModificationByOperation(String initSql, String sql, boolean commit, + int expSizeDelta) throws Exception { + checkSizeModificationByOperation( + cache -> cache.query(q(initSql)).getAll(), + cache -> cache.query(q(sql)).getAll(), + commit, + expSizeDelta); + } + + /** */ + private void checkSizeModificationByOperation(Consumer> inTx, boolean commit, + int expSizeDelta) throws Exception { + checkSizeModificationByOperation(c -> {}, inTx, commit, expSizeDelta); + } + + /** */ + private void checkSizeModificationByOperation(Consumer> beforeTx, + Consumer> inTx, boolean commit, int expSizeDelta) throws Exception { + IgniteCache tbl0 = grid(0).cache("person"); + + tbl0.query(q("delete from person")); + + beforeTx.accept(tbl0); + + int initSize = tbl0.size(); + + tbl0.query(q("begin")); + + inTx.accept(tbl0); + + // size is not changed before commit + assertEquals(0, tbl0.size() - initSize); + + if (commit) + tbl0.query(q("commit")); + else + tbl0.query(q("rollback")); + + assertEquals(expSizeDelta, tbl0.size() - initSize); + assertEquals(tbl0.size(), table(grid(1)).size()); + + assertEquals(tbl0.size(), tbl0.size(BACKUP)); + assertEquals(tbl0.size(), table(grid(1)).size(BACKUP)); + } + + /** + * @throws Exception if failed. + */ + public void testSql() throws Exception { + startGridsMultiThreaded(2); + + createTable(grid(0)); + + checkSizeModificationByOperation("insert into person values(1, 'a')", true, 1); + + checkSizeModificationByOperation("insert into person values(1, 'a')", false, 0); + + checkSizeModificationByOperation( + personTbl -> personTbl.query(q("insert into person values(1, 'a')")), + personTbl -> { + try { + personTbl.query(q("insert into person values(1, 'a')")); + } + catch (Exception e) { + if (e.getCause() instanceof IgniteSQLException) { + assertEquals(IgniteQueryErrorCode.DUPLICATE_KEY, + ((IgniteSQLException)e.getCause()).statusCode()); + } + else { + e.printStackTrace(); + + fail("Unexpected exceptions"); + } + } + }, + true, 0); + + checkSizeModificationByOperation("merge into person(id, name) values(1, 'a')", true, 1); + + checkSizeModificationByOperation("merge into person(id, name) values(1, 'a')", false, 0); + + checkSizeModificationByOperation( + "insert into person values(1, 'a')", "merge into person(id, name) values(1, 'b')", true, 0); + + checkSizeModificationByOperation("update person set name = 'b' where id = 1", true, 0); + + checkSizeModificationByOperation( + "insert into person values(1, 'a')", "update person set name = 'b' where id = 1", true, 0); + + checkSizeModificationByOperation( + "insert into person values(1, 'a')", "delete from person where id = 1", true, -1); + + checkSizeModificationByOperation( + "insert into person values(1, 'a')", "delete from person where id = 1", false, 0); + + checkSizeModificationByOperation("delete from person where id = 1", true, 0); + + checkSizeModificationByOperation( + "insert into person values(1, 'a')", "select * from person", true, 0); + + checkSizeModificationByOperation("select * from person", true, 0); + + checkSizeModificationByOperation( + "insert into person values(1, 'a')", "select * from person where id = 1 for update", true, 0); + + checkSizeModificationByOperation("select * from person where id = 1 for update", true, 0); + + checkSizeModificationByOperation(personTbl -> { + personTbl.query(q("insert into person values(1, 'a')")); + + personTbl.query(q("insert into person values(%d, 'b')", keyInSamePartition(grid(0), "person", 1))); + + personTbl.query(q("insert into person values(%d, 'c')", keyInDifferentPartition(grid(0), "person", 1))); + }, true, 3); + + checkSizeModificationByOperation(personTbl -> { + personTbl.query(q("insert into person values(1, 'a')")); + + personTbl.query(q("delete from person where id = 1")); + }, true, 0); + + checkSizeModificationByOperation(personTbl -> { + personTbl.query(q("insert into person values(1, 'a')")); + + personTbl.query(q("delete from person where id = 1")); + + personTbl.query(q("insert into person values(1, 'a')")); + }, true, 1); + + checkSizeModificationByOperation( + personTbl -> personTbl.query(q("insert into person values(1, 'a')")), + personTbl -> { + personTbl.query(q("delete from person where id = 1")); + + personTbl.query(q("insert into person values(1, 'a')")); + }, true, 0); + + checkSizeModificationByOperation(personTbl -> { + personTbl.query(q("merge into person(id, name) values(1, 'a')")); + + personTbl.query(q("delete from person where id = 1")); + }, true, 0); + + checkSizeModificationByOperation(personTbl -> { + personTbl.query(q("merge into person(id, name) values(1, 'a')")); + + personTbl.query(q("delete from person where id = 1")); + + personTbl.query(q("merge into person(id, name) values(1, 'a')")); + }, true, 1); + + checkSizeModificationByOperation( + personTbl -> personTbl.query(q("merge into person(id, name) values(1, 'a')")), + personTbl -> { + personTbl.query(q("delete from person where id = 1")); + + personTbl.query(q("merge into person(id, name) values(1, 'a')")); + }, true, 0); + } + + /** + * @throws Exception if failed. + */ + public void testInsertDeleteConcurrent() throws Exception { + startGridsMultiThreaded(2); + + IgniteCache tbl0 = createTable(grid(0)); + + SqlFieldsQuery insert = new SqlFieldsQuery("insert into person(id, name) values(?, 'a')"); + + SqlFieldsQuery delete = new SqlFieldsQuery("delete from person where id = ?"); + + CompletableFuture insertFut = CompletableFuture.supplyAsync(() -> { + int cnt = 0; + + for (int i = 0; i < 1000; i++) + cnt += update(insert.setArgs(ThreadLocalRandom.current().nextInt(10)), tbl0); + + return cnt; + }); + + CompletableFuture deleteFut = CompletableFuture.supplyAsync(() -> { + int cnt = 0; + + for (int i = 0; i < 1000; i++) + cnt += update(delete.setArgs(ThreadLocalRandom.current().nextInt(10)), tbl0); + + return cnt; + }); + + int expSize = insertFut.join() - deleteFut.join(); + + assertEquals(expSize, tbl0.size()); + assertEquals(expSize, table(grid(1)).size()); + + assertEquals(expSize, tbl0.size(BACKUP)); + assertEquals(expSize, table(grid(1)).size(BACKUP)); + } + + /** */ + private int update(SqlFieldsQuery qry, IgniteCache cache) { + try { + return Integer.parseInt(cache.query(qry).getAll().get(0).get(0).toString()); + } catch (Exception e) { + return 0; + } + } + + /** + * @throws Exception if failed. + */ + public void testWriteConflictDoesNotChangeSize() throws Exception { + startGridsMultiThreaded(2); + + IgniteCache tbl0 = createTable(grid(0)); + + tbl0.query(q("insert into person values(1, 'a')")); + + tbl0.query(q("begin")); + + tbl0.query(q("delete from person where id = 1")); + + CompletableFuture conflictingStarted = new CompletableFuture<>(); + + CompletableFuture fut = CompletableFuture.runAsync(() -> { + tbl0.query(q("begin")); + + try { + tbl0.query(q("select * from person")).getAll(); + conflictingStarted.complete(null); + + tbl0.query(q("merge into person(id, name) values(1, 'b')")); + } + finally { + tbl0.query(q("commit")); + } + }); + + conflictingStarted.join(); + tbl0.query(q("commit")); + + try { + fut.join(); + } + catch (Exception e) { + if (e.getCause().getCause() instanceof IgniteSQLException) + assertTrue(e.getMessage().toLowerCase().contains("version mismatch")); + else { + e.printStackTrace(); + + fail("Unexpected exception"); + } + } + + assertEquals(0, tbl0.size()); + assertEquals(0, table(grid(1)).size()); + + assertEquals(0, tbl0.size(BACKUP)); + assertEquals(0, table(grid(1)).size(BACKUP)); + } + + /** + * @throws Exception if failed. + */ + public void testDeleteChangesSizeAfterUnlock() throws Exception { + startGridsMultiThreaded(2); + + IgniteCache tbl0 = createTable(grid(0)); + + tbl0.query(q("insert into person values(1, 'a')")); + + tbl0.query(q("begin")); + + tbl0.query(q("select * from person where id = 1 for update")).getAll(); + + CompletableFuture asyncThread = new CompletableFuture<>(); + + CompletableFuture fut = CompletableFuture.runAsync(() -> { + tbl0.query(q("begin")); + + try { + tbl0.query(q("select * from person")).getAll(); + + asyncThread.complete(Thread.currentThread()); + tbl0.query(q("delete from person where id = 1")); + } + finally { + tbl0.query(q("commit")); + } + }); + + Thread concThread = asyncThread.join(); + + // wait until concurrent thread blocks awaiting entry mvcc lock release + while (concThread.getState() == Thread.State.RUNNABLE && !Thread.currentThread().isInterrupted()); + + tbl0.query(q("commit")); + + fut.join(); + + assertEquals(0, tbl0.size()); + assertEquals(0, table(grid(1)).size()); + + assertEquals(0, tbl0.size(BACKUP)); + assertEquals(0, table(grid(1)).size(BACKUP)); + } + + /** + * @throws Exception if failed. + */ + public void testDataStreamerModifiesReplicatedCacheSize() throws Exception { + startGridsMultiThreaded(2); + + IgniteEx ignite = grid(0); + + ignite.createCache( + new CacheConfiguration<>("test") + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setCacheMode(CacheMode.REPLICATED) + ); + + try (IgniteDataStreamer streamer = ignite.dataStreamer("test")) { + streamer.addData(1, "a"); + + streamer.addData(keyInDifferentPartition(ignite, "test", 1), "b"); + } + + assertEquals(2, ignite.cache("test").size()); + + assertEquals(1, grid(0).cache("test").localSize()); + assertEquals(1, grid(0).cache("test").localSize(BACKUP)); + + assertEquals(1, grid(1).cache("test").localSize()); + assertEquals(1, grid(1).cache("test").localSize(BACKUP)); + } + + /** + * @throws Exception if failed. + */ + public void testSizeIsConsistentAfterRebalance() throws Exception { + IgniteEx ignite = startGrid(0); + + IgniteCache tbl = createTable(ignite); + + for (int i = 0; i < 100; i++) + tbl.query(q("insert into person values(?, ?)").setArgs(i, i)); + + startGrid(1); + + awaitPartitionMapExchange(); + + IgniteCache tbl0 = grid(0).cache("person"); + IgniteCache tbl1 = grid(1).cache("person"); + + assert tbl0.localSize() != 0 && tbl1.localSize() != 0; + + assertEquals(100, tbl1.size()); + assertEquals(100, tbl0.localSize() + tbl1.localSize()); + } + + /** + * @throws Exception If failed. + */ + public void testSizeIsConsistentAfterRebalanceDuringInsert() throws Exception { + IgniteEx ignite = startGrid(0); + + IgniteCache tbl = createTable(ignite); + + Future f = null; + + for (int i = 0; i < 100; i++) { + if (i == 50) + f = ForkJoinPool.commonPool().submit(() -> startGrid(1)); + + tbl.query(q("insert into person values(?, ?)").setArgs(i, i)); + } + + f.get(); + + awaitPartitionMapExchange(); + + IgniteCache tbl0 = grid(0).cache("person"); + IgniteCache tbl1 = grid(1).cache("person"); + + assert tbl0.localSize() != 0 && tbl1.localSize() != 0; + + assertEquals(100, tbl1.size()); + assertEquals(100, tbl0.localSize() + tbl1.localSize()); + } + + /** */ + private static IgniteCache table(IgniteEx ignite) { + assert ignite.cachex("person").configuration().getAtomicityMode() == CacheAtomicityMode.TRANSACTIONAL; + assert ignite.cachex("person").configuration().getCacheMode() == CacheMode.REPLICATED; + + return ignite.cache("person"); + } + + /** */ + private static IgniteCache createTable(IgniteEx ignite) { + IgniteCache sqlNexus = ignite.getOrCreateCache(new CacheConfiguration<>("sqlNexus").setSqlSchema("PUBLIC")); + + sqlNexus.query(q("" + + "create table person(" + + " id int primary key," + + " name varchar" + + ") with \"atomicity=transactional,template=replicated,cache_name=person\"")); + + return table(ignite); + } + + /** */ + private static SqlFieldsQuery q(String fSql, Object... args) { + return new SqlFieldsQuery(String.format(fSql, args)); + } + + /** */ + private static int keyInSamePartition(Ignite ignite, String cacheName, int key) { + Affinity affinity = ignite.affinity(cacheName); + + return IntStream.iterate(key + 1, i -> i + 1) + .filter(i -> affinity.partition(i) == affinity.partition(key)) + .findFirst().getAsInt(); + } + + /** */ + private static int keyInDifferentPartition(Ignite ignite, String cacheName, int key) { + Affinity affinity = ignite.affinity(cacheName); + + return IntStream.iterate(key + 1, i -> i + 1) + .filter(i -> affinity.partition(i) != affinity.partition(key)) + .findFirst().getAsInt(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java index ab9d2e6763f3f..c8f7643a42fce 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSqlTxQueriesTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSqlTxQueriesWithReducerTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccScanQueryWithConcurrentJdbcTransactionTest; +import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentJdbcTransactionTest; import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccStreamingInsertTest; import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildWithMvccEnabledSelfTest; @@ -50,9 +51,10 @@ public class IgniteCacheMvccSqlTestSuite extends TestSuite { public static TestSuite suite() { TestSuite suite = new TestSuite("IgniteCache SQL MVCC Test Suite"); - // Simle tests. + // Simple tests. suite.addTestSuite(CacheMvccDmlSimpleTest.class); suite.addTestSuite(SqlTransactionsCommandsWithMvccEnabledSelfTest.class); + suite.addTestSuite(CacheMvccSizeTest.class); suite.addTestSuite(GridIndexRebuildWithMvccEnabledSelfTest.class); @@ -83,7 +85,6 @@ public static TestSuite suite() { suite.addTestSuite(CacheMvccPartitionedSqlCoordinatorFailoverTest.class); suite.addTestSuite(CacheMvccReplicatedSqlCoordinatorFailoverTest.class); - return suite; } } From c05e68df53132f0cd398cf2c1e80e33edaee38f9 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Fri, 7 Sep 2018 16:57:05 +0300 Subject: [PATCH 67/95] IGNITE-9498 SchemaExchangeSelfTest should use declared test IP finder - Fixes #4694. --- .../processors/cache/index/SchemaExchangeSelfTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java index b92c79289a333..c7709f2893955 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SchemaExchangeSelfTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; import java.util.Collections; @@ -47,6 +48,9 @@ public class SchemaExchangeSelfTest extends AbstractSchemaSelfTest { /** Node on which filter should be applied (if any). */ private static String filterNodeName; + /** */ + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); @@ -541,7 +545,7 @@ private IgniteEx start(int idx, boolean client, Class... clss) throws Exception cfg.setClientMode(client); cfg.setLocalHost("127.0.0.1"); - cfg.setDiscoverySpi(new TestTcpDiscoverySpi()); + cfg.setDiscoverySpi(new TestTcpDiscoverySpi().setIpFinder(IP_FINDER)); if (filterNodeName != null && F.eq(name, filterNodeName)) cfg.setUserAttributes(Collections.singletonMap("AFF_NODE", true)); @@ -591,7 +595,7 @@ private IgniteEx startNoCache(int idx, boolean client) throws Exception { cfg.setClientMode(client); cfg.setLocalHost("127.0.0.1"); - cfg.setDiscoverySpi(new TestTcpDiscoverySpi()); + cfg.setDiscoverySpi(new TestTcpDiscoverySpi().setIpFinder(IP_FINDER)); return (IgniteEx)Ignition.start(cfg); } From ef5833aa21c5d841843cb7de61ffb0f3acda07ad Mon Sep 17 00:00:00 2001 From: Ravil Galeyev Date: Fri, 7 Sep 2018 17:29:12 +0300 Subject: [PATCH 68/95] IGNITE-9285: [ML] Add MaxAbsScaler as a preprocessing stage this closes #4655 --- .../ml/preprocessing/MaxAbsScalerExample.java | 84 ++++++++++++++++++ .../ml/preprocessing/MinMaxScalerExample.java | 8 +- .../MaxAbsScalerPartitionData.java | 48 +++++++++++ .../MaxAbsScalerPreprocessor.java | 77 +++++++++++++++++ .../maxabsscaling/MaxAbsScalerTrainer.java | 85 +++++++++++++++++++ .../maxabsscaling/package-info.java | 22 +++++ .../MaxAbsScalerPreprocessorTest.java | 55 ++++++++++++ .../MaxAbsScalerTrainerTest.java | 76 +++++++++++++++++ 8 files changed, 451 insertions(+), 4 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MaxAbsScalerExample.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPartitionData.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessor.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/package-info.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessorTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainerTest.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MaxAbsScalerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MaxAbsScalerExample.java new file mode 100644 index 0000000000000..955702aaf07c9 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MaxAbsScalerExample.java @@ -0,0 +1,84 @@ +/* + * 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.examples.ml.preprocessing; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.examples.ml.dataset.model.Person; +import org.apache.ignite.examples.ml.util.DatasetHelper; +import org.apache.ignite.ml.dataset.DatasetFactory; +import org.apache.ignite.ml.dataset.primitive.SimpleDataset; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.apache.ignite.ml.preprocessing.maxabsscaling.MaxAbsScalerTrainer; + +/** + * Example that shows how to use MaxAbsScaler preprocessor to scale the given data. + * + * Machine learning preprocessors are built as a chain. Most often a first preprocessor is a feature extractor as shown + * in this example. The second preprocessor here is a MinMaxScaler preprocessor which is built on top of the feature + * extractor and represents a chain of itself and the underlying feature extractor. + */ +public class MaxAbsScalerExample { + /** Run example. */ + public static void main(String[] args) throws Exception { + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> Max abs example started."); + + IgniteCache persons = createCache(ignite); + + // Defines first preprocessor that extracts features from an upstream data. + IgniteBiFunction featureExtractor = (k, v) -> VectorUtils.of( + v.getAge(), + v.getSalary() + ); + + // Defines second preprocessor that processes features. + IgniteBiFunction preprocessor = new MaxAbsScalerTrainer() + .fit(ignite, persons, featureExtractor); + + // Creates a cache based simple dataset containing features and providing standard dataset API. + try (SimpleDataset dataset = DatasetFactory.createSimpleDataset(ignite, persons, preprocessor)) { + new DatasetHelper(dataset).describe(); + } + + System.out.println(">>> Max abs example completed."); + } + } + + /** */ + private static IgniteCache createCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + + cacheConfiguration.setName("PERSONS"); + cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 2)); + + IgniteCache persons = ignite.createCache(cacheConfiguration); + + persons.put(1, new Person("Mike", 42, 10000)); + persons.put(2, new Person("John", 32, 64000)); + persons.put(3, new Person("George", 53, 120000)); + persons.put(4, new Person("Karl", 24, 70000)); + + return persons; + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MinMaxScalerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MinMaxScalerExample.java index c1b32ab7951c8..f73228f75e03f 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MinMaxScalerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/MinMaxScalerExample.java @@ -42,8 +42,8 @@ *

      * After that it defines preprocessors that extract features from an upstream data and normalize their values.

      *

      - * Finally, it creates the dataset based on the processed data and uses Dataset API to find and output - * various statistical metrics of the data.

      + * Finally, it creates the dataset based on the processed data and uses Dataset API to find and output various + * statistical metrics of the data.

      *

      * You can change the test data used in this example and re-run it to explore this functionality further.

      */ @@ -51,7 +51,7 @@ public class MinMaxScalerExample { /** Run example. */ public static void main(String[] args) throws Exception { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> Normalization example started."); + System.out.println(">>> MinMax preprocessing example started."); IgniteCache persons = createCache(ignite); @@ -70,7 +70,7 @@ public static void main(String[] args) throws Exception { new DatasetHelper(dataset).describe(); } - System.out.println(">>> Normalization example completed."); + System.out.println(">>> MinMax preprocessing example completed."); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPartitionData.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPartitionData.java new file mode 100644 index 0000000000000..e4658da3d3a3f --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPartitionData.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.preprocessing.maxabsscaling; + +/** + * Partition data used in maxabsscaling preprocessor. + * + * @see MaxAbsScalerTrainer + * @see MaxAbsScalerPreprocessor + */ +public class MaxAbsScalerPartitionData implements AutoCloseable { + /** Maximum absolute values. */ + private final double[] maxAbs; + + /** + * Constructs a new instance of maxabsscaling partition data. + * + * @param maxAbs Maximal absolute values. + */ + public MaxAbsScalerPartitionData(double[] maxAbs) { + this.maxAbs = maxAbs; + } + + /** */ + public double[] getMaxAbs() { + return maxAbs; + } + + /** */ + @Override public void close() { + // Do nothing, GC will clean up. + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessor.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessor.java new file mode 100644 index 0000000000000..4eb0e31ca2771 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessor.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.ml.preprocessing.maxabsscaling; + +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; + +/** + * The preprocessing function that makes maxabsscaling, transforms features to the scale {@code [-1,+1]}. From + * mathematical point of view it's the following function which is applied to every element in a dataset: + * + * {@code a_i = a_i / maxabs_i for all i}, + * + * where {@code i} is a number of column, {@code maxabs_i} is the value of the absolute maximum element in this column. + * + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + */ +public class MaxAbsScalerPreprocessor implements IgniteBiFunction { + /** */ + private static final long serialVersionUID = 1L; + + /** Maximum absolute values. */ + private final double[] maxAbs; + + /** Base preprocessor. */ + private final IgniteBiFunction basePreprocessor; + + /** + * Constructs a new instance of maxabsscaling preprocessor. + * + * @param maxAbs Maximal absolute values. + * @param basePreprocessor Base preprocessor. + */ + public MaxAbsScalerPreprocessor(double[] maxAbs, IgniteBiFunction basePreprocessor) { + this.maxAbs = maxAbs; + this.basePreprocessor = basePreprocessor; + } + + /** + * Applies this preprocessor. + * + * @param k Key. + * @param v Value. + * @return Preprocessed row. + */ + @Override public Vector apply(K k, V v) { + Vector res = basePreprocessor.apply(k, v); + + assert res.size() == maxAbs.length; + + for (int i = 0; i < res.size(); i++) + res.set(i, res.get(i) / maxAbs[i]); + + return res; + } + + /** */ + public double[] getMaxAbs() { + return maxAbs; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainer.java new file mode 100644 index 0000000000000..d3e573471a07c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainer.java @@ -0,0 +1,85 @@ +/* + * 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.ml.preprocessing.maxabsscaling; + +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.UpstreamEntry; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.preprocessing.PreprocessingTrainer; + +/** + * Trainer of the maxabsscaling preprocessor. + * + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + */ +public class MaxAbsScalerTrainer implements PreprocessingTrainer { + /** {@inheritDoc} */ + @Override public MaxAbsScalerPreprocessor fit(DatasetBuilder datasetBuilder, + IgniteBiFunction basePreprocessor) { + try (Dataset dataset = datasetBuilder.build( + (upstream, upstreamSize) -> new EmptyContext(), + (upstream, upstreamSize, ctx) -> { + double[] maxAbs = null; + + while (upstream.hasNext()) { + UpstreamEntry entity = upstream.next(); + Vector row = basePreprocessor.apply(entity.getKey(), entity.getValue()); + + if (maxAbs == null) { + maxAbs = new double[row.size()]; + for (int i = 0; i < maxAbs.length; i++) + maxAbs[i] = .0; + } + else + assert maxAbs.length == row.size() : "Base preprocessor must return exactly " + maxAbs.length + + " features"; + + for (int i = 0; i < row.size(); i++) { + if (Math.abs(row.get(i)) > Math.abs(maxAbs[i])) + maxAbs[i] = Math.abs(row.get(i)); + } + } + return new MaxAbsScalerPartitionData(maxAbs); + } + )) { + double[] maxAbs = dataset.compute(MaxAbsScalerPartitionData::getMaxAbs, + (a, b) -> { + if (a == null) + return b; + + if (b == null) + return a; + + double[] result = new double[a.length]; + + for (int i = 0; i < result.length; i++) { + result[i] = Math.max(Math.abs(a[i]), Math.abs(b[i])); + } + return result; + }); + return new MaxAbsScalerPreprocessor<>(maxAbs, basePreprocessor); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/package-info.java new file mode 100644 index 0000000000000..343ba30810489 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/maxabsscaling/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 Max Abs Scaler preprocessor. + */ +package org.apache.ignite.ml.preprocessing.maxabsscaling; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessorTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessorTest.java new file mode 100644 index 0000000000000..3c30f3eda8b76 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerPreprocessorTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.preprocessing.maxabsscaling; + +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Test; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link MaxAbsScalerPreprocessor}. + */ +public class MaxAbsScalerPreprocessorTest { + /** Tests {@code apply()} method. */ + @Test + public void testApply() { + double[][] data = new double[][] { + {2., 4., 1.}, + {1., 8., 22.}, + {-4., 10., 100.}, + {0., 22., 300.} + }; + double[] maxAbs = new double[] {4, 22, 300}; + MaxAbsScalerPreprocessor preprocessor = new MaxAbsScalerPreprocessor<>( + maxAbs, + (k, v) -> v + ); + + double[][] expectedData = new double[][] { + {.5, 4. / 22, 1. / 300}, + {.25, 8. / 22, 22. / 300}, + {-1., 10. / 22, 100. / 300}, + {0., 22. / 22, 300. / 300} + }; + + for (int i = 0; i < data.length; i++) + assertArrayEquals(expectedData[i], preprocessor.apply(i, VectorUtils.of(data[i])).asArray(), 1e-8); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainerTest.java new file mode 100644 index 0000000000000..57116600244a6 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/maxabsscaling/MaxAbsScalerTrainerTest.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.ml.preprocessing.maxabsscaling; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.math.primitives.vector.Vector; +import org.apache.ignite.ml.math.primitives.vector.VectorUtils; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link MaxAbsScalerTrainer}. + */ +@RunWith(Parameterized.class) +public class MaxAbsScalerTrainerTest { + /** Parameters. */ + @Parameterized.Parameters(name = "Data divided on {0} partitions") + public static Iterable data() { + return Arrays.asList( + new Integer[] {1}, + new Integer[] {2}, + new Integer[] {3}, + new Integer[] {5}, + new Integer[] {7}, + new Integer[] {100}, + new Integer[] {1000} + ); + } + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + /** Tests {@code fit()} method. */ + @Test + public void testFit() { + Map data = new HashMap<>(); + data.put(1, VectorUtils.of(2, -4, 1)); + data.put(2, VectorUtils.of(1, -8, 22)); + data.put(3, VectorUtils.of(-4, 10, 100)); + data.put(4, VectorUtils.of(0, 22, 300)); + + DatasetBuilder datasetBuilder = new LocalDatasetBuilder<>(data, parts); + + MaxAbsScalerTrainer standardizationTrainer = new MaxAbsScalerTrainer<>(); + + MaxAbsScalerPreprocessor preprocessor = standardizationTrainer.fit( + datasetBuilder, + (k, v) -> v + ); + + assertArrayEquals(new double[] {4, 22, 300}, preprocessor.getMaxAbs(), 1e-8); + } +} From 0219e02b09e2ec96f51ea458535323ee60f19134 Mon Sep 17 00:00:00 2001 From: ibessonov Date: Fri, 7 Sep 2018 17:48:09 +0300 Subject: [PATCH 69/95] IGNITE-9472 Added permissions check on cluster activation - Fixes #4686. Signed-off-by: Dmitriy Govorukhin --- .../internal/processors/rest/GridRestProcessor.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java index ccb92df24dc89..b6c1310a19631 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java @@ -888,6 +888,12 @@ private void authorize(GridRestRequest req, SecurityContext sCtx) throws Securit break; + case CLUSTER_ACTIVE: + case CLUSTER_INACTIVE: + perm = SecurityPermission.ADMIN_OPS; + + break; + case CACHE_METRICS: case CACHE_SIZE: case CACHE_METADATA: @@ -901,8 +907,6 @@ private void authorize(GridRestRequest req, SecurityContext sCtx) throws Securit case NAME: case LOG: case CLUSTER_CURRENT_STATE: - case CLUSTER_ACTIVE: - case CLUSTER_INACTIVE: case AUTHENTICATE: case ADD_USER: case REMOVE_USER: From 5ebb988f22ac615aaf41947f1980ea4989683570 Mon Sep 17 00:00:00 2001 From: ibessonov Date: Fri, 7 Sep 2018 18:02:12 +0300 Subject: [PATCH 70/95] IGNITE-9475 Fix closures that has been created on client does not provide real class name to TASK_* permissions - Fixes #4688. Signed-off-by: Dmitriy Govorukhin --- .../internal/processors/task/GridTaskProcessor.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 2f0aa7b858940..9007472b034f5 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 @@ -537,8 +537,12 @@ private ComputeTaskInternalFuture startTask( String taskClsName; - if (task != null) - taskClsName = task.getClass().getName(); + if (task != null) { + if (task instanceof GridPeerDeployAware) + taskClsName = ((GridPeerDeployAware)task).deployClass().getName(); + else + taskClsName = task.getClass().getName(); + } else taskClsName = taskCls != null ? taskCls.getName() : taskName; From b20385933cf62694dfa15592bbb6b779fe5f5274 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Mon, 10 Sep 2018 16:43:10 +0700 Subject: [PATCH 71/95] IGNITE-7460 Web Console: Fixed issue with "step" param of "evictionThreshold" input. --- modules/web-console/frontend/app/services/Clusters.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/services/Clusters.js b/modules/web-console/frontend/app/services/Clusters.js index 0228a77b4d2cd..e0a2ec7ca40b4 100644 --- a/modules/web-console/frontend/app/services/Clusters.js +++ b/modules/web-console/frontend/app/services/Clusters.js @@ -266,7 +266,7 @@ export default class Clusters { } }, evictionThreshold: { - step: 0.05, + step: 0.001, max: 0.999, min: 0.5, default: 0.9 From 5da5b1edc981adabf54750c905bbbacd3a84f496 Mon Sep 17 00:00:00 2001 From: Sunny Chan Date: Mon, 10 Sep 2018 12:53:28 +0300 Subject: [PATCH 72/95] IGNITE-5960 Fixed race on continuous query registration and entry update. Fixes #2728. --- .../processors/cache/GridCacheMapEntry.java | 36 +++++++++++++++++ .../CacheContinuousQueryManager.java | 39 ++++++++++++++++--- 2 files changed, 69 insertions(+), 6 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 714d4a0e0719c..48067337f84e9 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 @@ -24,6 +24,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import javax.cache.Cache; import javax.cache.expiry.ExpiryPolicy; @@ -184,6 +185,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme @GridToStringExclude private final ReentrantLock lock = new ReentrantLock(); + /** Read Lock for continuous query listener */ + @GridToStringExclude + private final Lock listenerLock; + /** * Flags: *
        @@ -212,6 +217,7 @@ protected GridCacheMapEntry( this.key = key; this.hash = key.hashCode(); this.cctx = cctx; + this.listenerLock = cctx.continuousQueries().getListenerReadLock(); ver = GridCacheVersionManager.START_VER; } @@ -1371,6 +1377,7 @@ private void mvccDrReplicate(GridDrType drType, CacheObject val, GridCacheVersio GridLongList mvccWaitTxs = null; + lockListenerReadLock(); lockEntry(); try { @@ -1524,6 +1531,7 @@ else if (interceptorVal != val0) } finally { unlockEntry(); + unlockListenerReadLock(); } onUpdateFinished(updateCntr0); @@ -1602,6 +1610,7 @@ protected Object keyValue(boolean cpy) { GridLongList mvccWaitTxs = null; + lockListenerReadLock(); lockEntry(); try { @@ -1756,6 +1765,7 @@ else if (log.isDebugEnabled()) } finally { unlockEntry(); + unlockListenerReadLock(); } if (deferred) @@ -1818,6 +1828,7 @@ private boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) { EntryProcessorResult invokeRes = null; + lockListenerReadLock(); lockEntry(); try { @@ -2133,6 +2144,7 @@ else if (op == DELETE && transformOp) } finally { unlockEntry(); + unlockListenerReadLock(); } return new GridTuple3<>(res, @@ -2182,6 +2194,7 @@ else if (op == DELETE && transformOp) if (!primary && !isNear()) ensureFreeSpace(); + lockListenerReadLock(); lockEntry(); try { @@ -2443,6 +2456,7 @@ else if (updateMetrics && REMOVE_NO_VAL.equals(updateRes.outcome()) } finally { unlockEntry(); + unlockListenerReadLock(); } onUpdateFinished(c.updateRes.updateCounter()); @@ -3197,6 +3211,7 @@ protected final boolean hasValueUnlocked() { GridCacheVersion oldVer = null; + lockListenerReadLock(); lockEntry(); try { @@ -3380,6 +3395,7 @@ else if (deletedUnlocked()) } finally { unlockEntry(); + unlockListenerReadLock(); // It is necessary to execute these callbacks outside of lock to avoid deadlocks. @@ -4852,6 +4868,26 @@ private int extrasSize() { lock.unlock(); } + /** + * This method would obtain read lock for continuous query listener setup. This + * is to prevent race condition between entry update and continuous query setup. + * You should make sure you obtain this read lock first before locking the entry + * in order to ensure that the entry update is completed and existing continuous + * query notified before the next cache listener update + */ + private void lockListenerReadLock() { + listenerLock.lock(); + } + + /** + * unlock the listener read lock + * + * @see #lockListenerReadLock() + */ + private void unlockListenerReadLock() { + listenerLock.unlock(); + } + /** {@inheritDoc} */ @Override public boolean lockedByCurrentThread() { return lock.isHeldByCurrentThread(); 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 996e7f4c52ed5..0470fad5f6e25 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 @@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; import javax.cache.Cache; import javax.cache.configuration.CacheEntryListenerConfiguration; import javax.cache.configuration.Factory; @@ -64,6 +65,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.continuous.GridContinuousHandler; +import org.apache.ignite.internal.util.StripedCompositeReadWriteLock; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI2; @@ -126,6 +128,9 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter { /** Ordered topic prefix. */ private String topicPrefix; + /** ReadWriteLock to control the continuous query setup - this is to prevent the race between cache update and listener setup */ + private final StripedCompositeReadWriteLock listenerLock = new StripedCompositeReadWriteLock(Runtime.getRuntime().availableProcessors()) ; + /** Cancelable future task for backup cleaner */ private GridTimeoutProcessor.CancelableTask cancelableTask; @@ -194,6 +199,16 @@ protected GridTimeoutProcessor.CancelableTask getCancelableTask() { } } + /** + * Obtain the listener read lock, which must be held if any component need to + * read the list listener (generally caller to updateListener). + * + * @return Read lock for the listener update + */ + public Lock getListenerReadLock() { + return listenerLock.readLock(); + } + /** * @param lsnrs Listeners to notify. * @param key Entry key. @@ -884,9 +899,11 @@ private void advance() { * @param internal Internal flag. * @return Whether listener was actually registered. */ - GridContinuousHandler.RegisterStatus registerListener(UUID lsnrId, + GridContinuousHandler.RegisterStatus registerListener( + UUID lsnrId, CacheContinuousQueryListener lsnr, - boolean internal) { + boolean internal + ) { boolean added; if (internal) { @@ -896,7 +913,9 @@ GridContinuousHandler.RegisterStatus registerListener(UUID lsnrId, intLsnrCnt.incrementAndGet(); } else { - synchronized (this) { + listenerLock.writeLock().lock(); + + try { if (lsnrCnt.get() == 0) { if (cctx.group().sharedGroup() && !cctx.isLocal()) cctx.group().addCacheWithContinuousQuery(cctx); @@ -907,13 +926,16 @@ GridContinuousHandler.RegisterStatus registerListener(UUID lsnrId, if (added) lsnrCnt.incrementAndGet(); } + finally { + listenerLock.writeLock().unlock(); + } if (added) lsnr.onExecution(); } - return added ? GridContinuousHandler.RegisterStatus.REGISTERED : - GridContinuousHandler.RegisterStatus.NOT_REGISTERED; + return added ? GridContinuousHandler.RegisterStatus.REGISTERED + : GridContinuousHandler.RegisterStatus.NOT_REGISTERED; } /** @@ -931,7 +953,9 @@ void unregisterListener(boolean internal, UUID id) { } } else { - synchronized (this) { + listenerLock.writeLock().lock(); + + try { if ((lsnr = lsnrs.remove(id)) != null) { int cnt = lsnrCnt.decrementAndGet(); @@ -939,6 +963,9 @@ void unregisterListener(boolean internal, UUID id) { cctx.group().removeCacheWithContinuousQuery(cctx); } } + finally { + listenerLock.writeLock().unlock(); + } if (lsnr != null) lsnr.onUnregister(); From 2ca2bea754b3179844e14942d23931c8d764afd4 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Mon, 10 Sep 2018 14:33:40 +0300 Subject: [PATCH 73/95] IGNITE-8509 Fixed and reworkd tx rollback tests - Fixes #4150. Signed-off-by: Alexey Goncharuk --- .../processors/cache/GridCacheIoManager.java | 1 - .../transactions/TxRollbackAsyncTest.java | 255 ++++++++++-------- 2 files changed, 139 insertions(+), 117 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index 1e25c935f8104..2e66e5bfc3fe4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -611,7 +611,6 @@ private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, Grid } } - /** * @param cacheMsg Cache message. * @param nodeId Node ID. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java index 7968be31b9146..4ca8ba37c34ba 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache.transactions; -import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -25,7 +24,11 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -46,6 +49,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteFutureCancelledCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; @@ -55,6 +59,8 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.CIX1; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; @@ -81,6 +87,7 @@ import org.apache.ignite.transactions.TransactionIsolation; import org.apache.ignite.transactions.TransactionRollbackException; +import static java.lang.Thread.interrupted; import static java.lang.Thread.yield; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -344,13 +351,12 @@ private void testSynchronousRollback0(Ignite holdLockNode, final Ignite tryLockN CountDownLatch waitCommit = new CountDownLatch(1); + // Used for passing tx instance to rollback thread. IgniteInternalFuture lockFut = lockInTx(holdLockNode, keyLocked, waitCommit, 0); U.awaitQuiet(keyLocked); - final CountDownLatch rollbackLatch = new CountDownLatch(1); - - final int txCnt = 10000; + final int txCnt = 1000; final IgniteKernal k = (IgniteKernal)tryLockNode; @@ -358,7 +364,16 @@ private void testSynchronousRollback0(Ignite holdLockNode, final Ignite tryLockN final GridCacheContext cctx = ctx.cacheContext(CU.cacheId(CACHE_NAME)); - final AtomicBoolean stop = new AtomicBoolean(); + GridFutureAdapter txReadyFut = new GridFutureAdapter<>(); + + long seed = System.currentTimeMillis(); + + Random r = new Random(seed); + + log.info("Running: node0=" + holdLockNode.cluster().localNode().consistentId() + + ", node1=" + tryLockNode.cluster().localNode().consistentId() + + ", useTimeout=" + useTimeout + + ", seed=" + seed); IgniteInternalFuture txFut = multithreadedAsync(new Runnable() { @Override public void run() { @@ -369,10 +384,10 @@ private void testSynchronousRollback0(Ignite holdLockNode, final Ignite tryLockN assertTrue(tx0 == null || tx0.state() == ROLLED_BACK); - rollbackLatch.countDown(); - try (Transaction tx = tryLockNode.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, - useTimeout ? 500 : 0, 1)) { + useTimeout ? 50 : 0, 1)) { + + txReadyFut.onDone(tx); // Will block on lock request until rolled back asynchronously. Object o = tryLockNode.cache(CACHE_NAME).get(0); @@ -384,29 +399,30 @@ private void testSynchronousRollback0(Ignite holdLockNode, final Ignite tryLockN } } - stop.set(true); + txReadyFut.onDone((Transaction)null); } }, 1, "tx-get-thread"); IgniteInternalFuture rollbackFut = multithreadedAsync(new Runnable() { @Override public void run() { - U.awaitQuiet(rollbackLatch); - - doSleep(50); - Set rolledBackVers = new HashSet<>(); int proc = 1; - while(!stop.get()) { - for (Transaction tx : tryLockNode.transactions().localActiveTransactions()) { + while(true) { + try { + Transaction tx = txReadyFut.get(); + + txReadyFut.reset(); + + if (tx == null) + break; + + doSleep(r.nextInt(15)); // Wait a bit to reduce chance of rolling back empty transactions. + if (rolledBackVers.contains(tx.xid())) fail("Rollback version is expected"); - // Skip write transaction. - if (LABEL.equals(tx.label())) - continue; - try { if (proc % 2 == 0) tx.rollback(); @@ -419,14 +435,15 @@ private void testSynchronousRollback0(Ignite holdLockNode, final Ignite tryLockN rolledBackVers.add(tx.xid()); - if (proc % 1000 == 0) + if (proc % 100 == 0) log.info("Rolled back: " + proc); proc++; } + catch (IgniteCheckedException e) { + fail(e.getMessage()); + } } - - assertEquals("Unexpected size", txCnt, rolledBackVers.size()); } }, 1, "tx-rollback-thread"); @@ -613,8 +630,6 @@ public void testRollbackDelayFinishRequest() throws Exception { * */ public void testMixedAsyncRollbackTypes() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-8509"); - final Ignite client = startClient(); final AtomicBoolean stop = new AtomicBoolean(); @@ -640,134 +655,124 @@ public void testMixedAsyncRollbackTypes() throws Exception { final LongAdder failed = new LongAdder(); final LongAdder rolledBack = new LongAdder(); - IgniteInternalFuture txFut = multithreadedAsync(new Runnable() { - @Override public void run() { - while (!stop.get()) { - int nodeId = r.nextInt(GRID_CNT + 1); + ConcurrentMap> perNodeTxs = new ConcurrentHashMap<>(); - // Choose random node to start tx on. - Ignite node = nodeId == GRID_CNT || nearCacheEnabled() ? client : grid(nodeId); + for (Ignite ignite : G.allGrids()) + perNodeTxs.put(ignite, new ArrayBlockingQueue<>(1000)); - TransactionConcurrency conc = TC_VALS[r.nextInt(TC_VALS.length)]; - TransactionIsolation isolation = TI_VALS[r.nextInt(TI_VALS.length)]; + IgniteInternalFuture txFut = multithreadedAsync(() -> { + while (!stop.get()) { + int nodeId = r.nextInt(GRID_CNT + 1); - long timeout = r.nextInt(50) + 50; // Timeout is necessary to prevent deadlocks. + // Choose random node to start tx on. + Ignite node = nodeId == GRID_CNT || nearCacheEnabled() ? client : grid(nodeId); - try (Transaction tx = node.transactions().txStart(conc, isolation, timeout, txSize)) { - int setSize = r.nextInt(txSize / 2) + 1; + TransactionConcurrency conc = TC_VALS[r.nextInt(TC_VALS.length)]; + TransactionIsolation isolation = TI_VALS[r.nextInt(TI_VALS.length)]; - for (int i = 0; i < setSize; i++) { - switch (r.nextInt(4)) { - case 0: - node.cache(CACHE_NAME).remove(r.nextInt(txSize)); + // Timeout is necessary otherwise deadlock is possible due to randomness of lock acquisition. + long timeout = r.nextInt(50) + 50; - break; + try (Transaction tx = node.transactions().txStart(conc, isolation, timeout, txSize)) { + BlockingQueue nodeQ = perNodeTxs.get(node); - case 1: - node.cache(CACHE_NAME).get(r.nextInt(txSize)); + nodeQ.put(tx); - break; + int setSize = r.nextInt(txSize / 2) + 1; - case 2: - final Integer v = (Integer)node.cache(CACHE_NAME).get(r.nextInt(txSize)); + for (int i = 0; i < setSize; i++) { + switch (r.nextInt(4)) { + case 0: + node.cache(CACHE_NAME).remove(r.nextInt(txSize)); - node.cache(CACHE_NAME).put(r.nextInt(txSize), (v == null ? 0 : v) + 1); + break; - break; + case 1: + node.cache(CACHE_NAME).get(r.nextInt(txSize)); - case 3: - node.cache(CACHE_NAME).put(r.nextInt(txSize), 0); + break; - break; + case 2: + final Integer v = (Integer)node.cache(CACHE_NAME).get(r.nextInt(txSize)); - default: - fail("Unexpected opcode"); - } - } + node.cache(CACHE_NAME).put(r.nextInt(txSize), (v == null ? 0 : v) + 1); - tx.commit(); + break; - completed.add(1); - } - catch (Throwable e) { - failed.add(1); + case 3: + node.cache(CACHE_NAME).put(r.nextInt(txSize), 0); + + break; + + default: + fail("Unexpected opcode"); + } } - total.add(1); + tx.commit(); + + completed.add(1); } + catch (Throwable e) { + failed.add(1); + } + + total.add(1); } }, threadCnt, "tx-thread"); final AtomicIntegerArray idx = new AtomicIntegerArray(GRID_CNT + 1); - IgniteInternalFuture rollbackFut = multithreadedAsync(new Runnable() { - @Override public void run() { - int concurrentRollbackCnt = 5; - - List> futs = new ArrayList<>(concurrentRollbackCnt); - - while (!stop.get()) { - // Choose node randomly. - final int nodeId = r.nextInt(GRID_CNT + 1); - - // Reserve node. - if (!idx.compareAndSet(nodeId, 0, 1)) { - yield(); + CIX1 rollbackClo = new CIX1() { + @Override public void applyx(Transaction tx) throws IgniteCheckedException { + try { + IgniteFuture rollbackFut = tx.rollbackAsync(); - continue; - } + rollbackFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteFuture fut) { + tx.close(); + } + }); + } + catch (Throwable t) { + log.error("Exception on async rollback", t); - Ignite node = nodeId == GRID_CNT || nearCacheEnabled() ? client : grid(nodeId); + throw new IgniteCheckedException("Rollback failed", t); + } + } + }; - Collection transactions = node.transactions().localActiveTransactions(); + IgniteInternalFuture rollbackFut = multithreadedAsync(() -> { + while (!interrupted()) { + // Choose node randomly. + final int nodeId = r.nextInt(GRID_CNT + 1); - for (Transaction tx : transactions) { - rolledBack.add(1); + // Reserve node for rollback. + if (!idx.compareAndSet(nodeId, 0, 1)) { + yield(); - if (rolledBack.sum() % 1000 == 0) - info("Processed: " + rolledBack.sum()); + continue; + } - try { - IgniteFuture rollbackFut = tx.rollbackAsync(); + Ignite node = nodeId == GRID_CNT || nearCacheEnabled() ? client : grid(nodeId); - rollbackFut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteFuture fut) { - tx.close(); - } - }); + BlockingQueue nodeQ = perNodeTxs.get(node); - futs.add(rollbackFut); - } - catch (Throwable t) { - log.error("Exception on async rollback", t); + Transaction tx; - fail("Exception is not expected"); - } + // Rollback all transaction + while((tx = nodeQ.poll()) != null) { + rolledBack.add(1); - if (futs.size() == concurrentRollbackCnt) { - for (IgniteFuture fut : futs) - try { - fut.get(); - } - catch (IgniteException e) { - log.warning("Future was rolled back with error", e); - } + doSleep(r.nextInt(50)); // Add random sleep to increase completed txs count. - futs.clear(); - } - } + if (rolledBack.sum() % 1000 == 0) + info("Rolled back so far: " + rolledBack.sum()); - idx.set(nodeId, 0); + rollbackClo.apply(tx); } - for (IgniteFuture fut : futs) - try { - fut.get(); - } - catch (Throwable t) { - // No-op. - } - + idx.set(nodeId, 0); } }, 3, "rollback-thread"); // Rollback by multiple threads. @@ -775,9 +780,27 @@ public void testMixedAsyncRollbackTypes() throws Exception { stop.set(true); - txFut.get(); + txFut.get(); // Stop tx generation. - rollbackFut.get(); + rollbackFut.cancel(); + + try { + rollbackFut.get(); + } + catch (IgniteFutureCancelledCheckedException e) { + // Expected. + } + + // Rollback remaining transactions. + for (BlockingQueue queue : perNodeTxs.values()) { + Transaction tx; + + while((tx = queue.poll()) != null) { + rolledBack.add(1); + + rollbackClo.apply(tx); + } + } log.info("total=" + total.sum() + ", completed=" + completed.sum() + ", failed=" + failed.sum() + ", rolledBack=" + rolledBack.sum()); From a962a1ec7c5c149cc654b24a21ae3f427638bd5b Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Mon, 10 Sep 2018 22:14:34 +0700 Subject: [PATCH 74/95] IGNITE-8842 Web console: Fixed initial screen in demo mode. --- .../web-console/frontend/app/modules/demo/Demo.module.js | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/modules/web-console/frontend/app/modules/demo/Demo.module.js b/modules/web-console/frontend/app/modules/demo/Demo.module.js index 6ca19d6f8f43a..8cefcd51c9405 100644 --- a/modules/web-console/frontend/app/modules/demo/Demo.module.js +++ b/modules/web-console/frontend/app/modules/demo/Demo.module.js @@ -20,6 +20,8 @@ import angular from 'angular'; import DEMO_INFO from 'app/data/demo-info.json'; import templateUrl from 'views/templates/demo-info.tpl.pug'; +const DEMO_QUERY_STATE = {state: 'base.sql.notebook', params: {noteId: 'demo'}}; + angular .module('ignite-console.demo', [ 'ignite-console.socket' @@ -34,7 +36,7 @@ angular .state('demo.resume', { url: '/resume', permission: 'demo', - redirectTo: 'default-state', + redirectTo: DEMO_QUERY_STATE, unsaved: true, tfMetaTags: { title: 'Demo resume' @@ -47,11 +49,11 @@ angular const $http = trans.injector().get('$http'); return $http.post('/api/v1/demo/reset') - .then(() => 'default-state') + .then(() => DEMO_QUERY_STATE) .catch((err) => { trans.injector().get('IgniteMessages').showError(err); - return 'default-state'; + return DEMO_QUERY_STATE; }); }, unsaved: true, From c81c127ae719c9cd5b198927c6b4611107fbfbe7 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Mon, 10 Sep 2018 18:43:41 +0300 Subject: [PATCH 75/95] IGNITE-9445 Use valid tag for page write unlock while reading cold page from disk - Fixes #4708. Signed-off-by: Alexey Goncharuk --- .../persistence/pagemem/PageMemoryImpl.java | 7 +- .../CachePageWriteLockUnlockTest.java | 196 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite7.java | 3 + 3 files changed, 205 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePageWriteLockUnlockTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java index 75cc03632ffed..ba565c9f557d0 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java @@ -778,9 +778,13 @@ else if (relPtr == OUTDATED_REL_PTR) { ByteBuffer buf = wrapPointer(pageAddr, pageSize()); + long actualPageId = 0; + try { storeMgr.read(grpId, pageId, buf); + actualPageId = PageIO.getPageId(buf); + memMetrics.onPageRead(); } catch (IgniteDataIntegrityViolationException ignore) { @@ -794,7 +798,8 @@ else if (relPtr == OUTDATED_REL_PTR) { memMetrics.onPageRead(); } finally { - rwLock.writeUnlock(lockedPageAbsPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS); + rwLock.writeUnlock(lockedPageAbsPtr + PAGE_LOCK_OFFSET, + actualPageId == 0 ? OffheapReadWriteLock.TAG_LOCK_ALWAYS : PageIdUtils.tag(actualPageId)); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePageWriteLockUnlockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePageWriteLockUnlockTest.java new file mode 100644 index 0000000000000..f6a5ec1d1c20c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePageWriteLockUnlockTest.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.distributed; + +import java.util.Iterator; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; +import org.apache.ignite.internal.pagemem.store.PageStore; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +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.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; + +/** + * + */ +public class CachePageWriteLockUnlockTest extends GridCommonAbstractTest { + /** */ + public static final int PARTITION = 0; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME). + setAffinity(new RendezvousAffinityFunction(false, 32))); + + cfg.setActiveOnStart(false); + + DataStorageConfiguration memCfg = new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(DataStorageConfiguration.DFLT_DATA_REGION_INITIAL_SIZE) + ) + .setWalMode(WALMode.LOG_ONLY).setCheckpointFrequency(Integer.MAX_VALUE); + + cfg.setDataStorageConfiguration(memCfg); + + return cfg; + } + + /** + * + */ + public void testPreloadPartition() throws Exception { + try { + IgniteEx grid0 = startGrid(0); + + grid0.cluster().active(true); + + int total = 512; + + putData(grid0, total, PARTITION); + + grid0.cache(DEFAULT_CACHE_NAME).removeAll(); + + forceCheckpoint(); + + stopGrid(0); + + grid0 = startGrid(0); + + grid0.cluster().active(true); + + putData(grid0, total, PARTITION); // Will use pages from reuse pool. + + forceCheckpoint(); + + stopGrid(0); + + grid0 = startGrid(0); + + preloadPartition(grid0, DEFAULT_CACHE_NAME, PARTITION); + + Iterator> it = grid0.cache(DEFAULT_CACHE_NAME).iterator(); + + int c0 = 0; + + while (it.hasNext()) { + Cache.Entry entry = it.next(); + + c0++; + } + + assertEquals(total, c0); + } + finally { + stopAllGrids(); + } + } + + /** + * @param grid Grid. + * @param total Total. + * @param part Partition. + */ + private void putData(Ignite grid, int total, int part) { + int c = 0, k = 0; + + while(c < total) { + if (grid(0).affinity(DEFAULT_CACHE_NAME).partition(k) == part) { + grid.cache(DEFAULT_CACHE_NAME).put(k, k); + + c++; + } + + k++; + } + } + + /** + * Preload partition fast by iterating on all pages in disk order. + * + * @param grid Grid. + * @param cacheName Cache name. + * @param p P. + */ + private void preloadPartition(Ignite grid, String cacheName, int p) throws IgniteCheckedException { + GridDhtCacheAdapter dht = ((IgniteKernal)grid).internalCache(cacheName).context().dht(); + + GridDhtLocalPartition part = dht.topology().localPartition(p); + + assertNotNull(part); + + assertTrue(part.state() == OWNING); + + CacheGroupContext grpCtx = dht.context().group(); + + if (part.state() != OWNING) + return; + + IgnitePageStoreManager pageStoreMgr = grpCtx.shared().pageStore(); + + if (pageStoreMgr instanceof FilePageStoreManager) { + FilePageStoreManager filePageStoreMgr = (FilePageStoreManager)pageStoreMgr; + + PageStore pageStore = filePageStoreMgr.getStore(grpCtx.groupId(), part.id()); + + PageMemoryEx pageMemory = (PageMemoryEx)grpCtx.dataRegion().pageMemory(); + + long pageId = pageMemory.partitionMetaPageId(grpCtx.groupId(), part.id()); + + for (int pageNo = 0; pageNo < pageStore.pages(); pageId++, pageNo++) { + long pagePointer = -1; + + try { + pagePointer = pageMemory.acquirePage(grpCtx.groupId(), pageId); + } + finally { + if (pagePointer != -1) + pageMemory.releasePage(grpCtx.groupId(), pageId, pagePointer); + } + } + } + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + cleanPersistenceDir(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java index 0a2f86e45b705..6c48ecc2da6f7 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.WalModeChangeCoordinatorNotAffinityNodeSelfTest; import org.apache.ignite.internal.processors.cache.WalModeChangeSelfTest; import org.apache.ignite.internal.processors.cache.distributed.Cache64kPartitionsTest; +import org.apache.ignite.internal.processors.cache.distributed.CachePageWriteLockUnlockTest; import org.apache.ignite.internal.processors.cache.distributed.CacheRentingStateRepairTest; import org.apache.ignite.internal.processors.cache.distributed.CacheDataLossOnPartitionMoveTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheStartWithLoadTest; @@ -101,6 +102,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(TransactionIntegrityWithPrimaryIndexCorruptionTest.class); suite.addTestSuite(CacheDataLossOnPartitionMoveTest.class); + suite.addTestSuite(CachePageWriteLockUnlockTest.class); + return suite; } } From b1b132bb9b68f3aa76ab9fcd70324f94bb33c70f Mon Sep 17 00:00:00 2001 From: devozerov Date: Mon, 10 Sep 2018 22:55:55 +0300 Subject: [PATCH 76/95] Fixed a problem with optimized marshaller. --- .../TransactionConfiguration.java | 20 +++++ .../optimized/OptimizedClassDescriptor.java | 87 ++++++++++++------- .../service/GridServiceProcessor.java | 3 - .../processors/task/GridTaskWorker.java | 72 ++++++++------- .../internal/util/SerializableTransient.java | 10 +-- .../internal/util/TransientSerializable.java | 55 ++++++++++++ .../ignite/marshaller/MarshallerUtils.java | 22 ++++- 7 files changed, 197 insertions(+), 72 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/TransientSerializable.java diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java index 0ac215f2a4cee..e669bcf9d40a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java @@ -19,7 +19,9 @@ import java.io.Serializable; import javax.cache.configuration.Factory; +import org.apache.ignite.internal.util.TransientSerializable; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -27,7 +29,11 @@ /** * Transactions configuration. */ +@TransientSerializable(methodName = "transientSerializableFields") public class TransactionConfiguration implements Serializable { + /** */ + private static final IgniteProductVersion TX_PME_TIMEOUT_SINCE = IgniteProductVersion.fromString("2.5.1"); + /** */ private static final long serialVersionUID = 0L; @@ -387,4 +393,18 @@ public TransactionConfiguration setUseJtaSynchronization(boolean useJtaSync) { @Override public String toString() { return S.toString(TransactionConfiguration.class, this); } + + /** + * Excludes incompatible fields from serialization/deserialization process. + * + * @param ver Sender/Receiver node version. + * @return Array of excluded from serialization/deserialization fields. + */ + @SuppressWarnings("unused") + private static String[] transientSerializableFields(IgniteProductVersion ver) { + if (TX_PME_TIMEOUT_SINCE.compareToIgnoreTimestamp(ver) >= 0) + return new String[] { "txTimeoutOnPartitionMapExchange" }; + + return null; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedClassDescriptor.java index ccd99468a51db..0369b66513a10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedClassDescriptor.java @@ -43,16 +43,18 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.ConcurrentMap; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.SerializableTransient; +import org.apache.ignite.internal.util.TransientSerializable; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.MarshallerContext; import org.apache.ignite.marshaller.MarshallerExclusions; -import org.apache.ignite.marshaller.MarshallerUtils; import static java.lang.reflect.Modifier.isFinal; import static java.lang.reflect.Modifier.isPrivate; @@ -92,6 +94,8 @@ import static org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerUtils.STR; import static org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerUtils.UUID; import static org.apache.ignite.internal.marshaller.optimized.OptimizedMarshallerUtils.computeSerialVersionUid; +import static org.apache.ignite.marshaller.MarshallerUtils.jobReceiverVersion; +import static org.apache.ignite.marshaller.MarshallerUtils.jobSenderVersion; /** * Class descriptor. @@ -172,6 +176,9 @@ class OptimizedClassDescriptor { /** Method returns serializable transient fields. */ private Method serTransMtd; + /** Method returns transient serializable fields. */ + private Method transSerMtd; + /** * Creates descriptor for class. * @@ -448,16 +455,16 @@ else if (Proxy.class.isAssignableFrom(cls)) { readObjMtds.add(mtd); final SerializableTransient serTransAn = c.getAnnotation(SerializableTransient.class); + final TransientSerializable transSerAn = c.getAnnotation(TransientSerializable.class); // Custom serialization policy for transient fields. if (serTransAn != null) { try { - serTransMtd = c.getDeclaredMethod(serTransAn.methodName(), cls, IgniteProductVersion.class); + serTransMtd = c.getDeclaredMethod(serTransAn.methodName(), IgniteProductVersion.class); int mod = serTransMtd.getModifiers(); - if (isStatic(mod) && isPrivate(mod) - && serTransMtd.getReturnType() == String[].class) + if (isStatic(mod) && isPrivate(mod) && serTransMtd.getReturnType() == String[].class) serTransMtd.setAccessible(true); else // Set method back to null if it has incorrect signature. @@ -468,6 +475,24 @@ else if (Proxy.class.isAssignableFrom(cls)) { } } + // Custom serialization policy for non-transient fields. + if (transSerAn != null) { + try { + transSerMtd = c.getDeclaredMethod(transSerAn.methodName(), IgniteProductVersion.class); + + int mod = transSerMtd.getModifiers(); + + if (isStatic(mod) && isPrivate(mod) && transSerMtd.getReturnType() == String[].class) + transSerMtd.setAccessible(true); + else + // Set method back to null if it has incorrect signature. + transSerMtd = null; + } + catch (NoSuchMethodException ignored) { + transSerMtd = null; + } + } + Field[] clsFields0 = c.getDeclaredFields(); Map fieldNames = new HashMap<>(); @@ -824,7 +849,7 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { writeTypeData(out); out.writeShort(checksum); - out.writeSerializable(obj, writeObjMtds, serializableFields(obj.getClass(), obj, null)); + out.writeSerializable(obj, writeObjMtds, fields(obj.getClass(), jobReceiverVersion())); break; @@ -840,45 +865,52 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { * ignored. * * @param cls Class. - * @param obj Object. * @param ver Job sender version. * @return Serializable fields. */ @SuppressWarnings("ForLoopReplaceableByForEach") - private Fields serializableFields(Class cls, Object obj, IgniteProductVersion ver) { - if (serTransMtd == null) + private Fields fields(Class cls, IgniteProductVersion ver) { + if (ver == null // No context available. + || serTransMtd == null && transSerMtd == null) return fields; try { - final String[] transFields = (String[])serTransMtd.invoke(cls, obj, ver); + final String[] transFields = serTransMtd == null ? null : (String[])serTransMtd.invoke(null, ver); + final String[] serFields = transSerMtd == null ? null : (String[])transSerMtd.invoke(null, ver); - if (transFields == null || transFields.length == 0) + if (F.isEmpty(transFields) && F.isEmpty(serFields)) return fields; - List clsFields = new ArrayList<>(); + Map clsFields = new TreeMap<>(); - clsFields.addAll(fields.fields.get(0).fields); + for (FieldInfo field : fields.fields.get(0).fields) { + clsFields.put(field.fieldName, field); + } - for (int i = 0; i < transFields.length; i++) { - final String fieldName = transFields[i]; + // Add serializable transient fields + if (!F.isEmpty(transFields)) { + for (int i = 0; i < transFields.length; i++) { + final String fieldName = transFields[i]; - final Field f = cls.getDeclaredField(fieldName); + final Field f = cls.getDeclaredField(fieldName); - FieldInfo fieldInfo = new FieldInfo(f, f.getName(), - GridUnsafe.objectFieldOffset(f), fieldType(f.getType())); + FieldInfo fieldInfo = new FieldInfo(f, f.getName(), + GridUnsafe.objectFieldOffset(f), fieldType(f.getType())); - clsFields.add(fieldInfo); + clsFields.put(fieldName, fieldInfo); + } } - Collections.sort(clsFields, new Comparator() { - @Override public int compare(FieldInfo t1, FieldInfo t2) { - return t1.name().compareTo(t2.name()); + // Exclude non-transient fields which shouldn't be serialized. + if (!F.isEmpty(serFields)) { + for (int i = 0; i < serFields.length; i++) { + clsFields.remove(serFields[i]); } - }); + } - List fields = new ArrayList<>(); + List fields = new ArrayList<>(1); - fields.add(new ClassFields(clsFields)); + fields.add(new ClassFields(new ArrayList<>(clsFields.values()))); return new Fields(fields); } @@ -919,12 +951,7 @@ Object read(OptimizedObjectInputStream in) throws ClassNotFoundException, IOExce case SERIALIZABLE: verifyChecksum(in.readShort()); - // If no serialize method, then unmarshal as usual. - if (serTransMtd != null) - return in.readSerializable(cls, readObjMtds, readResolveMtd, - serializableFields(cls, null, MarshallerUtils.jobSenderVersion())); - else - return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); + return in.readSerializable(cls, readObjMtds, readResolveMtd, fields(cls, jobSenderVersion())); default: assert false : "Unexpected type: " + type; 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 9e8c7fad03163..fce705321dbd2 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 @@ -69,7 +69,6 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateAcceptedMessage; import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateProposedMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; @@ -79,7 +78,6 @@ 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; @@ -2127,7 +2125,6 @@ static class ServiceAssignmentsPredicate implements IgniteBiPredicate> { /** */ private static final long serialVersionUID = 0L; 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 82ad5d519cead..b73737c4827c2 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 @@ -84,6 +84,7 @@ import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.resources.TaskContinuousMapperResource; import org.jetbrains.annotations.Nullable; @@ -1375,38 +1376,45 @@ private void sendRequest(ComputeJobResult res) { boolean forceLocDep = internal || !ctx.deploy().enabled(); - req = new GridJobExecuteRequest( - ses.getId(), - res.getJobContext().getJobId(), - ses.getTaskName(), - ses.getUserVersion(), - ses.getTaskClassName(), - loc ? null : U.marshal(marsh, res.getJob()), - loc ? res.getJob() : null, - ses.getStartTime(), - timeout, - ses.getTopology(), - loc ? ses.getTopologyPredicate() : null, - loc ? null : U.marshal(marsh, ses.getTopologyPredicate()), - loc ? null : U.marshal(marsh, ses.getJobSiblings()), - loc ? ses.getJobSiblings() : null, - loc ? null : U.marshal(marsh, sesAttrs), - loc ? sesAttrs : null, - loc ? null : U.marshal(marsh, jobAttrs), - loc ? jobAttrs : null, - ses.getCheckpointSpi(), - dep.classLoaderId(), - dep.deployMode(), - continuous, - dep.participants(), - forceLocDep, - ses.isFullSupport(), - internal, - subjId, - affCacheIds, - affPartId, - mapTopVer, - ses.executorName()); + try { + MarshallerUtils.jobReceiverVersion(node.version()); + + req = new GridJobExecuteRequest( + ses.getId(), + res.getJobContext().getJobId(), + ses.getTaskName(), + ses.getUserVersion(), + ses.getTaskClassName(), + loc ? null : U.marshal(marsh, res.getJob()), + loc ? res.getJob() : null, + ses.getStartTime(), + timeout, + ses.getTopology(), + loc ? ses.getTopologyPredicate() : null, + loc ? null : U.marshal(marsh, ses.getTopologyPredicate()), + loc ? null : U.marshal(marsh, ses.getJobSiblings()), + loc ? ses.getJobSiblings() : null, + loc ? null : U.marshal(marsh, sesAttrs), + loc ? sesAttrs : null, + loc ? null : U.marshal(marsh, jobAttrs), + loc ? jobAttrs : null, + ses.getCheckpointSpi(), + dep.classLoaderId(), + dep.deployMode(), + continuous, + dep.participants(), + forceLocDep, + ses.isFullSupport(), + internal, + subjId, + affCacheIds, + affPartId, + mapTopVer, + ses.executorName()); + } + finally { + MarshallerUtils.jobReceiverVersion(null); + } if (loc) ctx.job().processJobExecuteRequest(ctx.discovery().localNode(), req); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java index 14a2f27b18829..e0160092b7fcd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java @@ -37,19 +37,17 @@ public @interface SerializableTransient { /** * Name of the private static method that returns list of transient fields - * that should be serialized (String[]), and accepts itself (before serialization) - * and {@link IgniteProductVersion}, e.g. + * that should be serialized (String[]), and accepts {@link IgniteProductVersion}, e.g. *
        -     *     private static String[] fields(Object self, IgniteProductVersion ver){
        +     *     private static String[] fields(IgniteProductVersion ver){
              *         return ver.compareTo("1.5.30") > 0 ? SERIALIZABLE_FIELDS : null;
              *     }
              * 
        *

        - * On serialization version argument ver is null, on deserialization - self is null. + * On serialization version argument ver is receiver version and sender version on deserialization. *

        *

        - * If it returns empty array or null all transient fields will be normally - * ignored. + * If it returns empty array or null all transient fields will be normally ignored. *

        * * @return Name of the method. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/TransientSerializable.java b/modules/core/src/main/java/org/apache/ignite/internal/util/TransientSerializable.java new file mode 100644 index 0000000000000..b583c1b48c5cd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/TransientSerializable.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.util; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; +import org.apache.ignite.lang.IgniteProductVersion; + +/** + * Marks class as it has non-transient fields that should not be serialized. + * Annotated class must have method that returns list of non-transient + * fields that should not be serialized. + *

        + * Works only for jobs. For other messages node version is not available. + *

        + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface TransientSerializable { + /** + * Name of the private static method that returns list of non-transient fields + * that should not be serialized (String[]), and accepts {@link IgniteProductVersion}, e.g. + *
        +     *     private static String[] fields(IgniteProductVersion ver){
        +     *         return ver.compareTo("1.5.30") < 0 ? EXCLUDED_FIELDS : null;
        +     *     }
        +     * 
        + *

        + * On serialization version argument ver is receiver version and sender version on deserialization. + *

        + *

        + * If it returns empty array or null all non-transient fields will be normally serialized. + *

        + * + * @return Name of the method. + */ + String methodName(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java index f7fef52973422..d5cf3865b595c 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java @@ -45,6 +45,9 @@ public class MarshallerUtils { /** Job sender node version. */ private static final ThreadLocal JOB_SND_NODE_VER = new ThreadLocal<>(); + /** Job sender node version. */ + private static final ThreadLocal JOB_RCV_NODE_VER = new ThreadLocal<>(); + /** * Set node name to marshaller context if possible. * @@ -95,6 +98,24 @@ public static IgniteProductVersion jobSenderVersion() { return JOB_SND_NODE_VER.get(); } + /** + * Sets thread local job receiver node version. + * + * @param ver Thread local job receiver node version. + */ + public static void jobReceiverVersion(IgniteProductVersion ver) { + JOB_RCV_NODE_VER.set(ver); + } + + /** + * Returns thread local job receiver node version. + * + * @return Thread local job receiver node version. + */ + public static IgniteProductVersion jobReceiverVersion() { + return JOB_RCV_NODE_VER.get(); + } + /** * Returns class name filter for marshaller. * @@ -199,5 +220,4 @@ private static void addClassNames( "[path=" + fileName + ']', e); } } - } From a17df67b47a6674a233d87d7bbab18960139ad72 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Tue, 11 Sep 2018 10:05:07 +0700 Subject: [PATCH 77/95] IGNITE-9528 Web Console: Fixed handling of "LOAD_COMPLETE_CONFIGURATION_ERR" action in PageConfigure service. Previous implementation used to throw a complete store message instead of an error it wraps. Also updated clusterServiceConfiguration generator to fail gracefully if some of caches do not exist. --- .../app/components/page-configure/services/PageConfigure.js | 2 +- .../modules/configuration/generator/ConfigurationGenerator.js | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-configure/services/PageConfigure.js b/modules/web-console/frontend/app/components/page-configure/services/PageConfigure.js index 10200bee1cdeb..35c1c013b2bed 100644 --- a/modules/web-console/frontend/app/components/page-configure/services/PageConfigure.js +++ b/modules/web-console/frontend/app/components/page-configure/services/PageConfigure.js @@ -55,7 +55,7 @@ export default class PageConfigure { .take(1) .do(() => this.ConfigureState.dispatchAction({type: 'LOAD_COMPLETE_CONFIGURATION', clusterID, isDemo})) .ignoreElements(), - this.ConfigureState.actions$.let(ofType('LOAD_COMPLETE_CONFIGURATION_ERR')).take(1).map((e) => {throw e;}), + this.ConfigureState.actions$.let(ofType('LOAD_COMPLETE_CONFIGURATION_ERR')).take(1).pluck('error').map((e) => Promise.reject(e)), this.ConfigureState.state$ .let(this.ConfigSelectors.selectCompleteClusterConfiguration({clusterID, isDemo})) .filter((c) => c.__isComplete) 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 c3efea5db8438..2a1a506d8b939 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -1698,7 +1698,7 @@ export default class IgniteConfigurationGenerator { .emptyBeanProperty('service') .intProperty('maxPerNodeCount') .intProperty('totalCount') - .stringProperty('cache', 'cacheName', (_id) => _id ? _.find(caches, {_id}).name : null) + .stringProperty('cache', 'cacheName', (_id) => _id ? _.get(_.find(caches, {_id}), 'name', null) : null) .stringProperty('affinityKey'); srvBeans.push(bean); From 07e4acc7ea779dc1a3b28f23b3db6be44d48bddb Mon Sep 17 00:00:00 2001 From: Alexey Platonov Date: Tue, 11 Sep 2018 11:39:54 +0300 Subject: [PATCH 78/95] IGNITE-8650 Fix flaky ZookeeperDiscovery testClientReconnect. - Fixes #4704. Signed-off-by: Dmitriy Govorukhin --- .../ignite/internal/IgniteClientReconnectCacheTest.java | 2 +- .../spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) 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 a975101758eb6..4df9b2622c242 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 @@ -1203,7 +1203,7 @@ private void reconnectMultinode(boolean longHist) throws Exception { ClusterGroup grp = client.cluster().forCacheNodes(DEFAULT_CACHE_NAME); - assertEquals(CLIENTS + srvNodes, grp.nodes().size()); + assertEquals(expNodes, grp.nodes().size()); grp = client.cluster().forClientNodes(DEFAULT_CACHE_NAME); diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java index 069b3e24951e7..2427c77275e83 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java @@ -1334,7 +1334,9 @@ private void checkClientsStatus(final List aliveNodes) throws Exception } } - assert !aliveClients.isEmpty(); + // This situation may appear while reconnection and this callback can be skipped. + if(!aliveClients.containsKey(locInternalOrder)) + return; Map.Entry oldest = aliveClients.firstEntry(); From 8297c89a4e191ca8742c6ca039ef4b97298d0ecb Mon Sep 17 00:00:00 2001 From: ibessonov Date: Tue, 11 Sep 2018 12:53:56 +0300 Subject: [PATCH 79/95] IGNITE-9518 Fix getPagesFillFactor returns NaN for empty region - Fixes #4717. Signed-off-by: Dmitriy Govorukhin --- .../persistence/DataRegionMetricsImpl.java | 4 +++- .../pagemem/FillFactorMetricTest.java | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java index 04617effe7e52..4334c7449b57a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java @@ -172,7 +172,9 @@ public DataRegionMetricsImpl(DataRegionConfiguration memPlcCfg, @Nullable Ignite long totalAllocated = getPageSize() * totalAllocatedPages.longValue(); - return (float) (totalAllocated - freeSpace) / totalAllocated; + return totalAllocated != 0 ? + (float) (totalAllocated - freeSpace) / totalAllocated + : 0f; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FillFactorMetricTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FillFactorMetricTest.java index 42eaf369692ec..ac65c6dcb2676 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FillFactorMetricTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FillFactorMetricTest.java @@ -90,6 +90,25 @@ protected CacheConfiguration cacheCfg() { */ private final float[] curFillFactor = new float[NODES]; + /** + * Tests that {@link DataRegionMetrics#getPagesFillFactor()} doesn't return NaN for empty cache. + * + * @throws Exception if failed. + */ + public void testEmptyCachePagesFillFactor() throws Exception { + startGrids(1); + + // Cache is created in default region so MY_DATA_REGION will have "empty" metrics. + CacheConfiguration cacheCfg = new CacheConfiguration<>().setName(MY_CACHE); + grid(0).getOrCreateCache(cacheCfg); + + DataRegionMetrics m = grid(0).dataRegionMetrics(MY_DATA_REGION); + + assertEquals(0, m.getTotalAllocatedPages()); + + assertEquals(0, m.getPagesFillFactor(), Float.MIN_VALUE); + } + /** * throws if failed. */ From 7c7cd6a3c54f34892063d49abfaadaa4bee2343f Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 14 Aug 2018 14:21:13 +0300 Subject: [PATCH 80/95] IGNITE-9141 Implemented --- .../h2/twostep/GridReduceQueryExecutor.java | 1 + .../query/h2/twostep/ReduceQueryRun.java | 1 + .../dht/MockGridDhtLocalPartition.java | 310 ++++++++++ .../h2/twostep/RetryCauseMessageTest.java | 560 ++++++++++++++++++ 4 files changed, 872 insertions(+) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java 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 96c88ffe7f4fb..8fbf4816ac94f 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 @@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import javax.cache.CacheException; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java index 7ddd653434daa..9f4b731298beb 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/ReduceQueryRun.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse; import org.h2.jdbc.JdbcConnection; import org.jetbrains.annotations.Nullable; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java new file mode 100644 index 0000000000000..33d7613c9c99a --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java @@ -0,0 +1,310 @@ +package org.apache.ignite.internal.processors.cache.distributed.dht; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.lang.IgniteInClosure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Wrapper around GridDhtLocalPartition to be extended in test cases + */ +public abstract class MockGridDhtLocalPartition extends GridDhtLocalPartition{ + /** + * fake id generator in order to bypass construction stage failure (if invoked with real id) + */ + private static AtomicInteger cntr = new AtomicInteger(Integer.MAX_VALUE); + + /** + * Real object + */ + private GridDhtLocalPartition internal; + + /** + * @param ctx Context. + * @param grp Cache group. + * @param id Partition ID. + */ + private MockGridDhtLocalPartition(GridCacheSharedContext ctx, + CacheGroupContext grp, int id) { + super(ctx, grp, id); + } + + /** */ + protected MockGridDhtLocalPartition(GridCacheSharedContext ctx, + CacheGroupContext grp, GridDhtLocalPartition internal){ + this(ctx, grp, cntr.getAndDecrement()); + this.internal = internal; + } + + /** */ + protected GridDhtLocalPartition getInternal(){ + return internal; + } + + /** {@inheritDoc} */ + @Override public int internalSize() { + return internal.internalSize(); + } + + /** {@inheritDoc} */ + @Override protected CacheMapHolder entriesMap(GridCacheContext cctx) { + return internal.entriesMap(cctx); + } + + /** {@inheritDoc} */ + @Nullable @Override protected CacheMapHolder entriesMapIfExists(Integer cacheId) { + return internal.entriesMapIfExists(cacheId); + } + + /** {@inheritDoc} */ + @Override public IgniteCacheOffheapManager.CacheDataStore dataStore() { + return internal.dataStore(); + } + + /** {@inheritDoc} */ + @Override public boolean addReservation(GridDhtPartitionsReservation r) { + return internal.addReservation(r); + } + + /** {@inheritDoc} */ + @Override public void removeReservation(GridDhtPartitionsReservation r) { + internal.removeReservation(r); + } + + /** {@inheritDoc} */ + @Override public int id() { + return internal.id(); + } + + /** {@inheritDoc} */ + @Override public GridDhtPartitionState state() { + return internal.state(); + } + + /** {@inheritDoc} */ + @Override public int reservations() { + return internal.reservations(); + } + + /** {@inheritDoc} */ + @Override public boolean isEmpty() { + return internal.isEmpty(); + } + + /** {@inheritDoc} */ + @Override public boolean valid() { + return internal.valid(); + } + + /** {@inheritDoc} */ + @Override public void cleanupRemoveQueue() { + internal.cleanupRemoveQueue(); + } + + /** {@inheritDoc} */ + @Override public void onDeferredDelete(int cacheId, KeyCacheObject key, GridCacheVersion ver) { + internal.onDeferredDelete(cacheId,key,ver); + } + + /** {@inheritDoc} */ + @Override public void lock() { + internal.lock(); + } + + /** {@inheritDoc} */ + @Override public void unlock() { + internal.unlock(); + } + + /** {@inheritDoc} */ + @Override public boolean reserve() { + return internal.reserve(); + } + + /** {@inheritDoc} */ + @Override public void release() { + internal.release(); + } + + /** {@inheritDoc} */ + @Override protected void release(int sizeChange, CacheMapHolder hld, GridCacheEntryEx e) { + internal.release(); + } + + /** {@inheritDoc} */ + @Override public void restoreState(GridDhtPartitionState stateToRestore) { + internal.restoreState(stateToRestore); + } + + /** {@inheritDoc} */ + @Override public void moving() { + + internal.moving(); + } + + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture rent(boolean updateSeq) { + return internal.rent(updateSeq); + } + + /** {@inheritDoc} */ + @Override public void clearAsync() { + internal.clearAsync(); + } + + /** {@inheritDoc} */ + @Override public boolean markForDestroy() { + return internal.markForDestroy(); + } + + /** {@inheritDoc} */ + @Override public void destroy() { + internal.destroy(); + } + + + /** {@inheritDoc} */ + @Override public void awaitDestroy() { + internal.awaitDestroy(); + } + + + /** {@inheritDoc} */ + @Override public void onClearFinished(IgniteInClosure> lsnr) { + internal.onClearFinished(lsnr); + } + + + /** {@inheritDoc} */ + @Override public boolean isClearing() { + return internal.isClearing(); + } + + + /** {@inheritDoc} */ + @Override public boolean tryClear(EvictionContext evictionCtx) throws NodeStoppingException { + return internal.tryClear(evictionCtx); + } + + /** {@inheritDoc} */ + @Override public boolean primary(AffinityTopologyVersion topVer) { + return internal.primary(topVer); + } + + + /** {@inheritDoc} */ + @Override public boolean backup(AffinityTopologyVersion topVer) { + return internal.backup(topVer); + } + + /** {@inheritDoc} */ + @Override public long initialUpdateCounter() { + return internal.initialUpdateCounter(); + } + + + /** {@inheritDoc} */ + @Override public void updateCounter(long val) { + + internal.updateCounter(val); + } + + + /** {@inheritDoc} */ + @Override public void initialUpdateCounter(long val) { + + internal.initialUpdateCounter(val); + } + + + /** {@inheritDoc} */ + @Override public long fullSize() { + + return internal.fullSize(); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + + return internal.hashCode(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + + return internal.equals(obj); + } + + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull GridDhtLocalPartition part) { + return internal.compareTo(part); + } + + /** {@inheritDoc} */ + @Override public String toString() { + + return internal.toString(); + } + + /** {@inheritDoc} */ + @Override public int publicSize(int cacheId) { + + return internal.publicSize(cacheId); + } + + /** {@inheritDoc} */ + @Override public void incrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { + internal.incrementPublicSize(hld,e); + } + + /** {@inheritDoc} */ + @Override public void decrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { + internal.decrementPublicSize(hld,e); + } + + /** {@inheritDoc} */ + @Nullable @Override public GridCacheMapEntry getEntry(GridCacheContext ctx, KeyCacheObject key) { + return internal.getEntry(ctx,key); + } + + /** {@inheritDoc} */ + @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent( + GridCacheContext ctx, + final AffinityTopologyVersion topVer, + KeyCacheObject key, + final boolean create, + final boolean touch) { + return internal.putEntryIfObsoleteOrAbsent(ctx, topVer, key, create, touch); + } + + /** {@inheritDoc} */ + @Override public boolean removeEntry(final GridCacheEntryEx entry) { + + return internal.removeEntry(entry); + } + + /** {@inheritDoc} */ + @Override public Collection entries(int cacheId, final CacheEntryPredicate... filter) { + return internal.entries(cacheId, filter); + } + + /** {@inheritDoc} */ + @Override public Set entrySet(int cacheId, final CacheEntryPredicate... filter) { + return internal.entrySet(cacheId, filter); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java new file mode 100644 index 0000000000000..570d568fee212 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java @@ -0,0 +1,560 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceArray; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; +import org.apache.ignite.internal.processors.cache.distributed.dht.MockGridDhtLocalPartition; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +import org.apache.ignite.internal.util.GridSpinBusyLock; +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.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; + +/** + * Test for 6 retry cases + */ +public class RetryCauseMessageTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int NODES_COUNT = 2; + + /** */ + private static final int ORG_COUNT = NODES_COUNT; + + /** */ + private static final int PERSON_PER_ORG_COUNT = 50; + /** */ + private static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + + "where Person.orgId = org.id " + + "and lower(org.name) = lower(?)"; + /** */ + private static final String ORG_SQL = "select * from Organization"; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + /** */ + private IgniteCache orgCache; + /** */ + private IgniteH2Indexing h2Idx; + + /** */ + @Override protected long getTestTimeout() { + return 600 * 1000; + } + + /** + * Failed to reserve partitions for query (cache is not found on local node) + */ + public void testCacheWasNotFoundMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + qryReq.caches().add(Integer.MAX_VALUE); + startedExecutor.onMessage(nodeId, msg); + qryReq.caches().remove(qryReq.caches().size() - 1); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (group reservation failed) + */ + public void testGrpReservationFailureMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + return false; + } + @Override public void release() {} + + }); + } + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (group reservation failed) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) + */ + public void testReplicatedCacheReserveFailureMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); + } + else + startedExecutor.onMessage(nodeId, msg); + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery<>(Organization.class, ORG_SQL); + qry.setDistributedJoins(true); + try { + orgCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) + */ + public void testPartitionedCacheReserveFailureMessage() { + GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); + AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); + long stateVal = aState.getAndSet(2); + startedExecutor.onMessage(nodeId, msg); + aState.getAndSet(stateVal); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) + */ + public void testPartitionStateChangedMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + return true; + } + + @Override public void release() { + + } + }); + GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); + + if (GridDhtPartitionTopologyImpl.class.isAssignableFrom(cctx.topology().getClass())) { + GridDhtPartitionTopologyImpl tpg = (GridDhtPartitionTopologyImpl)(cctx.topology()); + AtomicReferenceArray locParts = GridTestUtils.getFieldValue(tpg, GridDhtPartitionTopologyImpl.class, "locParts"); + GridDhtLocalPartition part = locParts.get(0); + MockGridDhtLocalPartition mockPart = new MockGridDhtLocalPartition( + GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "ctx"), + GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "grp"), + part) { + volatile private boolean preReserved = true; + + @Override public boolean reserve() { + preReserved = false; + return super.reserve(); + } + + @Override public GridDhtPartitionState state() { + if (preReserved) + return super.state(); + else + return MOVING; + } + }; + locParts.set(0, mockPart); + startedExecutor.onMessage(nodeId, msg); + locParts.set(0, part); + } + else + startedExecutor.onMessage(nodeId, msg); + + } + else + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** + * Failed to execute non-collocated query (will retry) + */ + public void testNonCollocatedFailureMessage() { + final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); + final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", + new MockGridMapQueryExecutor(null) { + @Override public void onMessage(UUID nodeId, Object msg) { + if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { + final MapReservationKey grpKey = new MapReservationKey(ORG, null); + reservations.put(grpKey, new GridReservable() { + @Override public boolean reserve() { + throw new GridH2RetryException("test retry exception"); + } + + @Override public void release() { + + } + }); + } + startedExecutor.onMessage(nodeId, msg); + + } + }.insertRealExecutor(mapQryExec)); + + SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + } + catch (CacheException e) { + assertTrue(e.getMessage().contains("Failed to execute non-collocated query (will retry) [")); + return; + } + finally { + GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); + } + fail(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setPeerClassLoadingEnabled(false); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + disco.setForceServerMode(true); + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); + GridQueryProcessor qryProc = grid(ignite.name()).context().query(); + + h2Idx = GridTestUtils.getFieldValue(qryProc, GridQueryProcessor.class, "idx"); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setIndexedTypes(String.class, Person.class)); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, Organization.class) + ); + + awaitPartitionMapExchange(); + + populateDataIntoOrg(orgCache); + populateDataIntoPerson(personCache); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * Populate organization cache with test data + * @param cache @{IgniteCache} + */ + private void populateDataIntoOrg(IgniteCache cache) { + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + cache.put(org.getId(), org); + } + } + + /** + * Populate person cache with test data + * @param cache @{IgniteCache} + */ + private void populateDataIntoPerson(IgniteCache cache) { + int personId = 0; + for (int i = 0; i < ORG_COUNT; i++) { + Organization org = new Organization(); + org.setId("org" + i); + org.setName("Organization #" + i); + + for (int j = 0; j < PERSON_PER_ORG_COUNT; j++) { + Person prsn = new Person(); + prsn.setId("pers" + personId); + prsn.setOrgId(org.getId()); + prsn.setName("Person name #" + personId); + + cache.put(prsn.getId(), prsn); + + personId++; + } + } + } + + /** + * + */ + private static class Person { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String orgId; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public String getId() { + return id; + } + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getOrgId() { + return orgId; + } + + /** */ + public void setOrgId(String orgId) { + this.orgId = orgId; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + + /** + * + */ + private static class Organization { + /** */ + @QuerySqlField(index = true) + private String id; + + /** */ + @QuerySqlField(index = true) + private String name; + + /** */ + public void setId(String id) { + this.id = id; + } + + /** */ + public String getId() { + return id; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public void setName(String name) { + this.name = name; + } + } + + /** + * Wrapper around @{GridMapQueryExecutor} + */ + private abstract static class MockGridMapQueryExecutor extends GridMapQueryExecutor { + + /** + * Wrapped executor + */ + GridMapQueryExecutor startedExecutor; + + /** */ + MockGridMapQueryExecutor insertRealExecutor(GridMapQueryExecutor realExecutor) { + this.startedExecutor = realExecutor; + return this; + } + + /** + * @param busyLock Busy lock. + */ + MockGridMapQueryExecutor(GridSpinBusyLock busyLock) { + super(busyLock); + } + + /** {@inheritDoc} */ + @Override public void onMessage(UUID nodeId, Object msg) { + + startedExecutor.onMessage(nodeId, msg); + } + + /** {@inheritDoc} */ + @Override public void cancelLazyWorkers() { + + startedExecutor.cancelLazyWorkers(); + } + + /** {@inheritDoc} */ + @Override GridSpinBusyLock busyLock() { + + return startedExecutor.busyLock(); + + } + + /** {@inheritDoc} */ + @Override public void onCacheStop(String cacheName) { + + startedExecutor.onCacheStop(cacheName); + } + + /** {@inheritDoc} */ + @Override public void stopAndUnregisterCurrentLazyWorker() { + startedExecutor.stopAndUnregisterCurrentLazyWorker(); + } + + /** {@inheritDoc} */ + @Override public void unregisterLazyWorker(MapQueryLazyWorker worker) { + startedExecutor.unregisterLazyWorker(worker); + } + + /** {@inheritDoc} */ + @Override public int registeredLazyWorkers() { + + return startedExecutor.registeredLazyWorkers(); + } + } + +} From 59cc5937f61089b9b7a4d8a9f0e670cd1187897e Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 14 Aug 2018 14:21:13 +0300 Subject: [PATCH 81/95] IGNITE-9141 Implemented --- .../processors/query/h2/twostep/GridReduceQueryExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8fbf4816ac94f..1f2a36ab86f22 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 @@ -300,7 +300,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { * @param node Node. * @param msg Message. */ - private void onNextPage(final ClusterNode node, final GridQueryNextPageResponse msg) { + private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { final long qryReqId = msg.queryRequestId(); final int qry = msg.query(); final int seg = msg.segmentId(); From 0cb72c95606cff75b84803b7b1d24b90d036a6ae Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Mon, 20 Aug 2018 14:56:11 +0300 Subject: [PATCH 82/95] IGNITE-9141 Modified according to review comments. Integration tests added --- ...DisappearedCacheCauseRetryMessageTest.java | 106 +++++++++++ ...isappearedCacheWasNotFoundMessageTest.java | 96 ++++++++++ .../NonCollocatedRetryMessageTest.java | 112 ++++++++++++ .../h2/twostep/RetryCauseMessageTest.java | 168 +++--------------- 4 files changed, 340 insertions(+), 142 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java new file mode 100644 index 0000000000000..0072fe4a0a79a --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java @@ -0,0 +1,106 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import javax.cache.CacheException; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheMode; +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.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +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.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; + +/** + * Failed to reserve partitions for query (cache is not found on local node) Root cause test + */ +public class DisappearedCacheCauseRetryMessageTest extends GridCommonAbstractTest { + + /** */ + private static final int NODES_COUNT = 2; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + /** */ + private IgniteCache orgCache; + + public void testDisappearedCacheCauseRetryMessage() { + + SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + fail("No CacheException emitted."); + } + catch (CacheException e) { + assertTrue(e.getMessage(), e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + } + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + + volatile long reqId = -1; + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ + GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + reqId = req.requestId(); + orgCache.destroy(); + } else if ( GridQueryCancelRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridQueryCancelRequest req = (GridQueryCancelRequest) (gridMsg.message()); + + if (reqId == req.queryRequestId()) + orgCache = DisappearedCacheCauseRetryMessageTest.this.ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) + ); + + } + } + super.sendMessage(node, msg, ackC); + } + }); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + startGridsMultiThreaded(NODES_COUNT, false); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) + ); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) + ); + + awaitPartitionMapExchange(); + + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); + } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java new file mode 100644 index 0000000000000..ce106a8136877 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java @@ -0,0 +1,96 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheMode; +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.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +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.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; + +/** + * Grid cache context is not registered for cache id root cause message test + */ +public class DisappearedCacheWasNotFoundMessageTest extends GridCommonAbstractTest { + + /** */ + private static final int NODES_COUNT = 2; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + /** */ + private IgniteCache orgCache; + + public void testDisappearedCacheWasNotFoundMessage() { + + SqlQuery qry = new SqlQuery(Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + personCache.query(qry).getAll(); + fail("No CacheException emitted."); + } + catch (CacheException e) { + assertTrue(e.getMessage(), e.getMessage().contains("Grid cache context is not registered for cache id")); + } + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ + GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + req.requestId(); + orgCache.destroy(); + } + } + super.sendMessage(node, msg, ackC); + } + }); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) + ); + orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setCacheMode(CacheMode.REPLICATED) + .setIndexedTypes(String.class, Organization.class) + ); + + awaitPartitionMapExchange(); + + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java new file mode 100644 index 0000000000000..62058b42260dd --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java @@ -0,0 +1,112 @@ +package org.apache.ignite.internal.processors.query.h2.twostep; + +import java.util.List; +import javax.cache.Cache; +import javax.cache.CacheException; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; +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.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; +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.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; + +/** + * Failed to execute non-collocated query root cause message test + */ +public class NonCollocatedRetryMessageTest extends GridCommonAbstractTest { + /** */ + private static final int NODES_COUNT = 3; + /** */ + private static final String ORG = "org"; + /** */ + private IgniteCache personCache; + + /** */ + public void testNonCollocatedRetryMessage() { + SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); + qry.setDistributedJoins(true); + try { + List> prsns = personCache.query(qry).getAll(); + fail("No CacheException emitted. Collection size="+prsns.size()); + } + catch (CacheException e) { + assertTrue(e.getMessage(), e.getMessage().contains("Failed to execute non-collocated query")); + } + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + volatile long reqId = -1; + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; + if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ + GridIoMessage gridMsg = (GridIoMessage)msg; + if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ + GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); + if ( reqId < 0 ) { + reqId = req.requestId(); + String shutName = getTestIgniteInstanceName(1); + stopGrid(shutName, true, false); + } else if( reqId != req.requestId() ){ + try { + U.sleep(IgniteSystemProperties.getLong(IGNITE_SQL_RETRY_TIMEOUT, GridReduceQueryExecutor.DFLT_RETRY_TIMEOUT)); + } + catch (IgniteInterruptedCheckedException e) { + // no-op + } + } + } + } + super.sendMessage(node, msg, ackC); + } + }); + cfg.setDiscoverySpi(new TcpDiscoverySpi(){ + public long getNodesJoined() { + return stats.joinedNodesCount(); + } + }); + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); + startGridsMultiThreaded(NODES_COUNT, false); + + personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") + .setBackups(1) + .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) + ); + final IgniteCache orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) + .setBackups(1) + .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) + ); + + awaitPartitionMapExchange(); + + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); + } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + +} + diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java index 570d568fee212..815f9b0d06d38 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java @@ -7,9 +7,10 @@ import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.query.SqlQuery; -import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; @@ -24,14 +25,19 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; import org.apache.ignite.internal.util.GridSpinBusyLock; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +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.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; - +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.JOIN_SQL; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; +import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; /** * Test for 6 retry cases */ @@ -41,16 +47,6 @@ public class RetryCauseMessageTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 2; - - /** */ - private static final int ORG_COUNT = NODES_COUNT; - - /** */ - private static final int PERSON_PER_ORG_COUNT = 50; - /** */ - private static final String JOIN_SQL = "select * from Person, \"org\".Organization as org " + - "where Person.orgId = org.id " + - "and lower(org.name) = lower(?)"; /** */ private static final String ORG_SQL = "select * from Organization"; /** */ @@ -70,7 +66,7 @@ public class RetryCauseMessageTest extends GridCommonAbstractTest { /** * Failed to reserve partitions for query (cache is not found on local node) */ - public void testCacheWasNotFoundMessage() { + public void testSynthCacheWasNotFoundMessage() { GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", new MockGridMapQueryExecutor(null) { @@ -81,9 +77,9 @@ public void testCacheWasNotFoundMessage() { startedExecutor.onMessage(nodeId, msg); qryReq.caches().remove(qryReq.caches().size() - 1); } - else + else startedExecutor.onMessage(nodeId, msg); - + } }.insertRealExecutor(mapQryExec)); @@ -93,7 +89,7 @@ public void testCacheWasNotFoundMessage() { personCache.query(qry).getAll(); } catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); + assertTrue(e.getMessage(), e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); return; } finally { @@ -339,35 +335,39 @@ public void testNonCollocatedFailureMessage() { @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setPeerClassLoadingEnabled(false); + cfg.setCommunicationSpi(new TcpCommunicationSpi(){ + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + assert msg != null; - TcpDiscoverySpi disco = new TcpDiscoverySpi(); - disco.setForceServerMode(true); - disco.setIpFinder(IP_FINDER); - - cfg.setDiscoverySpi(disco); + super.sendMessage(node, msg, ackC); + } + }); return cfg; } /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); GridQueryProcessor qryProc = grid(ignite.name()).context().query(); h2Idx = GridTestUtils.getFieldValue(qryProc, GridQueryProcessor.class, "idx"); personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") - .setIndexedTypes(String.class, Person.class)); + .setIndexedTypes(String.class, Person.class) + ); orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) .setCacheMode(CacheMode.REPLICATED) .setIndexedTypes(String.class, Organization.class) + // .setNodeFilter( node -> node.order()<2L ) ); awaitPartitionMapExchange(); - populateDataIntoOrg(orgCache); - populateDataIntoPerson(personCache); + JoinSqlTestHelper.populateDataIntoOrg(orgCache); + JoinSqlTestHelper.populateDataIntoPerson(personCache); } /** {@inheritDoc} */ @@ -375,122 +375,6 @@ public void testNonCollocatedFailureMessage() { stopAllGrids(); } - /** - * Populate organization cache with test data - * @param cache @{IgniteCache} - */ - private void populateDataIntoOrg(IgniteCache cache) { - for (int i = 0; i < ORG_COUNT; i++) { - Organization org = new Organization(); - org.setId("org" + i); - org.setName("Organization #" + i); - cache.put(org.getId(), org); - } - } - - /** - * Populate person cache with test data - * @param cache @{IgniteCache} - */ - private void populateDataIntoPerson(IgniteCache cache) { - int personId = 0; - for (int i = 0; i < ORG_COUNT; i++) { - Organization org = new Organization(); - org.setId("org" + i); - org.setName("Organization #" + i); - - for (int j = 0; j < PERSON_PER_ORG_COUNT; j++) { - Person prsn = new Person(); - prsn.setId("pers" + personId); - prsn.setOrgId(org.getId()); - prsn.setName("Person name #" + personId); - - cache.put(prsn.getId(), prsn); - - personId++; - } - } - } - - /** - * - */ - private static class Person { - /** */ - @QuerySqlField(index = true) - private String id; - - /** */ - @QuerySqlField(index = true) - private String orgId; - - /** */ - @QuerySqlField(index = true) - private String name; - - /** */ - public String getId() { - return id; - } - - /** */ - public void setId(String id) { - this.id = id; - } - - /** */ - public String getOrgId() { - return orgId; - } - - /** */ - public void setOrgId(String orgId) { - this.orgId = orgId; - } - - /** */ - public String getName() { - return name; - } - - /** */ - public void setName(String name) { - this.name = name; - } - } - - /** - * - */ - private static class Organization { - /** */ - @QuerySqlField(index = true) - private String id; - - /** */ - @QuerySqlField(index = true) - private String name; - - /** */ - public void setId(String id) { - this.id = id; - } - - /** */ - public String getId() { - return id; - } - - /** */ - public String getName() { - return name; - } - - /** */ - public void setName(String name) { - this.name = name; - } - } /** * Wrapper around @{GridMapQueryExecutor} From be238f4ab71f956071f4a464def7d929adc1c932 Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 28 Aug 2018 11:45:55 +0300 Subject: [PATCH 83/95] Review comments. --- .../twostep/messages/GridQueryNextPageResponse.java | 2 +- .../distributed/dht/MockGridDhtLocalPartition.java | 1 + .../h2/twostep/NonCollocatedRetryMessageTest.java | 11 +++++++++-- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java index aff90e495244e..6b976c24179cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java @@ -426,4 +426,4 @@ public boolean removeMapping() { "valsSize", vals != null ? vals.size() : 0, "rowsSize", plainRows != null ? plainRows.size() : 0); } -} +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java index 33d7613c9c99a..2a0dcbe449ef2 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java @@ -22,6 +22,7 @@ /** * Wrapper around GridDhtLocalPartition to be extended in test cases */ +// TODO: Move to H2 tests public abstract class MockGridDhtLocalPartition extends GridDhtLocalPartition{ /** * fake id generator in order to bypass construction stage failure (if invoked with real id) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java index 62058b42260dd..b0aa3840842f0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java @@ -25,11 +25,15 @@ /** * Failed to execute non-collocated query root cause message test */ +// TODO: Test -> SelfTest +// TODO: Add to suite. public class NonCollocatedRetryMessageTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 3; + /** */ private static final String ORG = "org"; + /** */ private IgniteCache personCache; @@ -58,11 +62,14 @@ public void testNonCollocatedRetryMessage() { GridIoMessage gridMsg = (GridIoMessage)msg; if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); - if ( reqId < 0 ) { + if (reqId < 0) { reqId = req.requestId(); + String shutName = getTestIgniteInstanceName(1); + stopGrid(shutName, true, false); - } else if( reqId != req.requestId() ){ + } + else if( reqId != req.requestId() ){ try { U.sleep(IgniteSystemProperties.getLong(IGNITE_SQL_RETRY_TIMEOUT, GridReduceQueryExecutor.DFLT_RETRY_TIMEOUT)); } From 87d200abd3defc7b85b9041870733f50fcda6285 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 28 Aug 2018 14:08:47 +0300 Subject: [PATCH 84/95] IGNITE-9141 Fixes according to review --- .../dht/MockGridDhtLocalPartition.java | 311 ------------ ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 92 ++-- ...DisappearedCacheCauseRetryMessageTest.java | 106 ----- ...pearedCacheWasNotFoundMessageSelfTest.java | 2 +- ...isappearedCacheWasNotFoundMessageTest.java | 96 ---- .../query/h2/twostep/JoinSqlTestHelper.java | 1 - .../NonCollocatedRetryMessageTest.java | 119 ----- .../h2/twostep/RetryCauseMessageSelfTest.java | 8 +- .../h2/twostep/RetryCauseMessageTest.java | 444 ------------------ 9 files changed, 34 insertions(+), 1145 deletions(-) delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java delete mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java deleted file mode 100644 index 2a0dcbe449ef2..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/MockGridDhtLocalPartition.java +++ /dev/null @@ -1,311 +0,0 @@ -package org.apache.ignite.internal.processors.cache.distributed.dht; - -import java.util.Collection; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.NodeStoppingException; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; -import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; -import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; -import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.lang.IgniteInClosure; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -/** - * Wrapper around GridDhtLocalPartition to be extended in test cases - */ -// TODO: Move to H2 tests -public abstract class MockGridDhtLocalPartition extends GridDhtLocalPartition{ - /** - * fake id generator in order to bypass construction stage failure (if invoked with real id) - */ - private static AtomicInteger cntr = new AtomicInteger(Integer.MAX_VALUE); - - /** - * Real object - */ - private GridDhtLocalPartition internal; - - /** - * @param ctx Context. - * @param grp Cache group. - * @param id Partition ID. - */ - private MockGridDhtLocalPartition(GridCacheSharedContext ctx, - CacheGroupContext grp, int id) { - super(ctx, grp, id); - } - - /** */ - protected MockGridDhtLocalPartition(GridCacheSharedContext ctx, - CacheGroupContext grp, GridDhtLocalPartition internal){ - this(ctx, grp, cntr.getAndDecrement()); - this.internal = internal; - } - - /** */ - protected GridDhtLocalPartition getInternal(){ - return internal; - } - - /** {@inheritDoc} */ - @Override public int internalSize() { - return internal.internalSize(); - } - - /** {@inheritDoc} */ - @Override protected CacheMapHolder entriesMap(GridCacheContext cctx) { - return internal.entriesMap(cctx); - } - - /** {@inheritDoc} */ - @Nullable @Override protected CacheMapHolder entriesMapIfExists(Integer cacheId) { - return internal.entriesMapIfExists(cacheId); - } - - /** {@inheritDoc} */ - @Override public IgniteCacheOffheapManager.CacheDataStore dataStore() { - return internal.dataStore(); - } - - /** {@inheritDoc} */ - @Override public boolean addReservation(GridDhtPartitionsReservation r) { - return internal.addReservation(r); - } - - /** {@inheritDoc} */ - @Override public void removeReservation(GridDhtPartitionsReservation r) { - internal.removeReservation(r); - } - - /** {@inheritDoc} */ - @Override public int id() { - return internal.id(); - } - - /** {@inheritDoc} */ - @Override public GridDhtPartitionState state() { - return internal.state(); - } - - /** {@inheritDoc} */ - @Override public int reservations() { - return internal.reservations(); - } - - /** {@inheritDoc} */ - @Override public boolean isEmpty() { - return internal.isEmpty(); - } - - /** {@inheritDoc} */ - @Override public boolean valid() { - return internal.valid(); - } - - /** {@inheritDoc} */ - @Override public void cleanupRemoveQueue() { - internal.cleanupRemoveQueue(); - } - - /** {@inheritDoc} */ - @Override public void onDeferredDelete(int cacheId, KeyCacheObject key, GridCacheVersion ver) { - internal.onDeferredDelete(cacheId,key,ver); - } - - /** {@inheritDoc} */ - @Override public void lock() { - internal.lock(); - } - - /** {@inheritDoc} */ - @Override public void unlock() { - internal.unlock(); - } - - /** {@inheritDoc} */ - @Override public boolean reserve() { - return internal.reserve(); - } - - /** {@inheritDoc} */ - @Override public void release() { - internal.release(); - } - - /** {@inheritDoc} */ - @Override protected void release(int sizeChange, CacheMapHolder hld, GridCacheEntryEx e) { - internal.release(); - } - - /** {@inheritDoc} */ - @Override public void restoreState(GridDhtPartitionState stateToRestore) { - internal.restoreState(stateToRestore); - } - - /** {@inheritDoc} */ - @Override public void moving() { - - internal.moving(); - } - - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture rent(boolean updateSeq) { - return internal.rent(updateSeq); - } - - /** {@inheritDoc} */ - @Override public void clearAsync() { - internal.clearAsync(); - } - - /** {@inheritDoc} */ - @Override public boolean markForDestroy() { - return internal.markForDestroy(); - } - - /** {@inheritDoc} */ - @Override public void destroy() { - internal.destroy(); - } - - - /** {@inheritDoc} */ - @Override public void awaitDestroy() { - internal.awaitDestroy(); - } - - - /** {@inheritDoc} */ - @Override public void onClearFinished(IgniteInClosure> lsnr) { - internal.onClearFinished(lsnr); - } - - - /** {@inheritDoc} */ - @Override public boolean isClearing() { - return internal.isClearing(); - } - - - /** {@inheritDoc} */ - @Override public boolean tryClear(EvictionContext evictionCtx) throws NodeStoppingException { - return internal.tryClear(evictionCtx); - } - - /** {@inheritDoc} */ - @Override public boolean primary(AffinityTopologyVersion topVer) { - return internal.primary(topVer); - } - - - /** {@inheritDoc} */ - @Override public boolean backup(AffinityTopologyVersion topVer) { - return internal.backup(topVer); - } - - /** {@inheritDoc} */ - @Override public long initialUpdateCounter() { - return internal.initialUpdateCounter(); - } - - - /** {@inheritDoc} */ - @Override public void updateCounter(long val) { - - internal.updateCounter(val); - } - - - /** {@inheritDoc} */ - @Override public void initialUpdateCounter(long val) { - - internal.initialUpdateCounter(val); - } - - - /** {@inheritDoc} */ - @Override public long fullSize() { - - return internal.fullSize(); - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - - return internal.hashCode(); - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object obj) { - - return internal.equals(obj); - } - - /** {@inheritDoc} */ - @Override public int compareTo(@NotNull GridDhtLocalPartition part) { - return internal.compareTo(part); - } - - /** {@inheritDoc} */ - @Override public String toString() { - - return internal.toString(); - } - - /** {@inheritDoc} */ - @Override public int publicSize(int cacheId) { - - return internal.publicSize(cacheId); - } - - /** {@inheritDoc} */ - @Override public void incrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { - internal.incrementPublicSize(hld,e); - } - - /** {@inheritDoc} */ - @Override public void decrementPublicSize(@Nullable CacheMapHolder hld, GridCacheEntryEx e) { - internal.decrementPublicSize(hld,e); - } - - /** {@inheritDoc} */ - @Nullable @Override public GridCacheMapEntry getEntry(GridCacheContext ctx, KeyCacheObject key) { - return internal.getEntry(ctx,key); - } - - /** {@inheritDoc} */ - @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent( - GridCacheContext ctx, - final AffinityTopologyVersion topVer, - KeyCacheObject key, - final boolean create, - final boolean touch) { - return internal.putEntryIfObsoleteOrAbsent(ctx, topVer, key, create, touch); - } - - /** {@inheritDoc} */ - @Override public boolean removeEntry(final GridCacheEntryEx entry) { - - return internal.removeEntry(entry); - } - - /** {@inheritDoc} */ - @Override public Collection entries(int cacheId, final CacheEntryPredicate... filter) { - return internal.entries(cacheId, filter); - } - - /** {@inheritDoc} */ - @Override public Set entrySet(int cacheId, final CacheEntryPredicate... filter) { - return internal.entrySet(cacheId, filter); - } -} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index d05a33e3c39c9..a0cea727cc6b0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -35,7 +35,6 @@ import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; -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.spi.discovery.tcp.TcpDiscoverySpi; @@ -47,7 +46,7 @@ */ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends GridCommonAbstractTest { /** Grids count. */ - private static final int GRIDS_CNT = 4; + private static final int GRIDS_CNT = 3; /** IP finder. */ private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -67,12 +66,6 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr /** */ private static final String QRY_3 = "select a._val from String a"; - /** */ - private static final String CANCELLED_BY_CLIENT = "reason=Cancelled by client"; - - /** */ - private static final String WITH_TIMEOUT_WAS_CANCELLED = "reason=Statement with timeout was cancelled"; - /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); @@ -107,96 +100,82 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr /** */ public void testRemoteQueryExecutionTimeout() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, true, - WITH_TIMEOUT_WAS_CANCELLED); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, true); } /** */ - public void testRemoteQueryWithMergeTableTimeout0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.MILLISECONDS, true, - WITH_TIMEOUT_WAS_CANCELLED); - } - - /** Query possibly could be executed faster than timeout*/ - public void testRemoteQueryWithMergeTableTimeout1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 25, TimeUnit.MILLISECONDS, true, - WITH_TIMEOUT_WAS_CANCELLED); + public void testRemoteQueryWithMergeTableTimeout() throws Exception { + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, true); } /** */ public void testRemoteQueryExecutionCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false, - CANCELLED_BY_CLIENT); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryExecutionCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, false, - CANCELLED_BY_CLIENT); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryExecutionCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false, - CANCELLED_BY_CLIENT); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false); } /** */ public void testRemoteQueryExecutionCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false, - CANCELLED_BY_CLIENT); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false); } /** */ public void testRemoteQueryWithMergeTableCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false, - CANCELLED_BY_CLIENT); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** */ public void testRemoteQueryWithMergeTableCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false, null); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** */ public void testRemoteQueryWithMergeTableCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false, null); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** */ public void testRemoteQueryWithMergeTableCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false, null); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false); } - /** Query possibly could be executed faster than timeout*/ + /** */ public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { - testQueryCancel(2*CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false, - CANCELLED_BY_CLIENT); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** */ public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false, null); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** */ public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false, null); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** */ public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** */ public void testRemoteQueryAlreadyFinishedStop() throws Exception { - testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); + testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); } /** */ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, - boolean timeout, String cause) throws Exception { + boolean timeout) throws Exception { try (Ignite client = startGrid("client")) { IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); @@ -237,25 +216,18 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni try(QueryCursor> ignored = cursor) { cursor.iterator(); - - if (!F.isEmpty(cause)) - fail("No exception caught"); + fail(); } catch (CacheException ex) { - log().error("Got exception", ex); - - log().error( "Cause of exception", ex.getCause()); + log().error("Got expected exception", ex); assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException); + } - assertTrue( "Cause message "+ex.getCause().getMessage(), ex.getCause().getMessage().contains(cause)); - }finally { - - // Give some time to clean up. - Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); + // Give some time to clean up. + Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); - checkCleanState(); - } + checkCleanState(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java deleted file mode 100644 index 0072fe4a0a79a..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheCauseRetryMessageTest.java +++ /dev/null @@ -1,106 +0,0 @@ -package org.apache.ignite.internal.processors.query.h2.twostep; - -import javax.cache.CacheException; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cache.CacheMode; -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.internal.managers.communication.GridIoMessage; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; -import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; -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.testframework.junits.common.GridCommonAbstractTest; - -import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; -import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; -import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; - -/** - * Failed to reserve partitions for query (cache is not found on local node) Root cause test - */ -public class DisappearedCacheCauseRetryMessageTest extends GridCommonAbstractTest { - - /** */ - private static final int NODES_COUNT = 2; - /** */ - private static final String ORG = "org"; - /** */ - private IgniteCache personCache; - /** */ - private IgniteCache orgCache; - - public void testDisappearedCacheCauseRetryMessage() { - - SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - fail("No CacheException emitted."); - } - catch (CacheException e) { - assertTrue(e.getMessage(), e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); - } - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setCommunicationSpi(new TcpCommunicationSpi(){ - - volatile long reqId = -1; - /** {@inheritDoc} */ - @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { - assert msg != null; - if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ - GridIoMessage gridMsg = (GridIoMessage)msg; - if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ - GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); - reqId = req.requestId(); - orgCache.destroy(); - } else if ( GridQueryCancelRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ - GridQueryCancelRequest req = (GridQueryCancelRequest) (gridMsg.message()); - - if (reqId == req.queryRequestId()) - orgCache = DisappearedCacheCauseRetryMessageTest.this.ignite(0).getOrCreateCache(new CacheConfiguration(ORG) - .setCacheMode(CacheMode.REPLICATED) - .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) - ); - - } - } - super.sendMessage(node, msg, ackC); - } - }); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); - startGridsMultiThreaded(NODES_COUNT, false); - - personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") - .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) - ); - orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) - .setCacheMode(CacheMode.REPLICATED) - .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) - ); - - awaitPartitionMapExchange(); - - JoinSqlTestHelper.populateDataIntoOrg(orgCache); - JoinSqlTestHelper.populateDataIntoPerson(personCache); - } - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - } -} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java index 9928ed6ff2745..91af36e72f85c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageSelfTest.java @@ -62,7 +62,7 @@ public void testDisappearedCacheWasNotFoundMessage() { fail("No CacheException emitted."); } catch (CacheException e) { - assertTrue(e.getMessage(), e.getMessage().contains("Cache not found on local node")); + assertTrue(e.getMessage(), e.getMessage().contains("Grid cache context is not registered for cache id")); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java deleted file mode 100644 index ce106a8136877..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/DisappearedCacheWasNotFoundMessageTest.java +++ /dev/null @@ -1,96 +0,0 @@ -package org.apache.ignite.internal.processors.query.h2.twostep; - -import javax.cache.CacheException; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cache.CacheMode; -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.internal.managers.communication.GridIoMessage; -import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; -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.testframework.junits.common.GridCommonAbstractTest; - -import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; -import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; -import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; - -/** - * Grid cache context is not registered for cache id root cause message test - */ -public class DisappearedCacheWasNotFoundMessageTest extends GridCommonAbstractTest { - - /** */ - private static final int NODES_COUNT = 2; - /** */ - private static final String ORG = "org"; - /** */ - private IgniteCache personCache; - /** */ - private IgniteCache orgCache; - - public void testDisappearedCacheWasNotFoundMessage() { - - SqlQuery qry = new SqlQuery(Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - fail("No CacheException emitted."); - } - catch (CacheException e) { - assertTrue(e.getMessage(), e.getMessage().contains("Grid cache context is not registered for cache id")); - } - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setCommunicationSpi(new TcpCommunicationSpi(){ - /** {@inheritDoc} */ - @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { - assert msg != null; - if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ - GridIoMessage gridMsg = (GridIoMessage)msg; - if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ - GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); - req.requestId(); - orgCache.destroy(); - } - } - super.sendMessage(node, msg, ackC); - } - }); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); - Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); - - personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") - .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) - ); - orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) - .setCacheMode(CacheMode.REPLICATED) - .setIndexedTypes(String.class, Organization.class) - ); - - awaitPartitionMapExchange(); - - JoinSqlTestHelper.populateDataIntoOrg(orgCache); - JoinSqlTestHelper.populateDataIntoPerson(personCache); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - } -} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java index fe7821ae29709..f0b8ce71cc1c0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/JoinSqlTestHelper.java @@ -26,7 +26,6 @@ public class JoinSqlTestHelper { /** */ private static final int ORG_COUNT = 100; - /** */ private static final int PERSON_PER_ORG_COUNT = 10; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java deleted file mode 100644 index b0aa3840842f0..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/NonCollocatedRetryMessageTest.java +++ /dev/null @@ -1,119 +0,0 @@ -package org.apache.ignite.internal.processors.query.h2.twostep; - -import java.util.List; -import javax.cache.Cache; -import javax.cache.CacheException; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteSystemProperties; -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.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.communication.GridIoMessage; -import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; -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.communication.tcp.TcpCommunicationSpi; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; - -/** - * Failed to execute non-collocated query root cause message test - */ -// TODO: Test -> SelfTest -// TODO: Add to suite. -public class NonCollocatedRetryMessageTest extends GridCommonAbstractTest { - /** */ - private static final int NODES_COUNT = 3; - - /** */ - private static final String ORG = "org"; - - /** */ - private IgniteCache personCache; - - /** */ - public void testNonCollocatedRetryMessage() { - SqlQuery qry = new SqlQuery(JoinSqlTestHelper.Person.class, JoinSqlTestHelper.JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - List> prsns = personCache.query(qry).getAll(); - fail("No CacheException emitted. Collection size="+prsns.size()); - } - catch (CacheException e) { - assertTrue(e.getMessage(), e.getMessage().contains("Failed to execute non-collocated query")); - } - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setCommunicationSpi(new TcpCommunicationSpi(){ - volatile long reqId = -1; - /** {@inheritDoc} */ - @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { - assert msg != null; - if ( GridIoMessage.class.isAssignableFrom(msg.getClass())){ - GridIoMessage gridMsg = (GridIoMessage)msg; - if ( GridH2QueryRequest.class.isAssignableFrom( gridMsg.message().getClass() ) ){ - GridH2QueryRequest req = (GridH2QueryRequest) (gridMsg.message()); - if (reqId < 0) { - reqId = req.requestId(); - - String shutName = getTestIgniteInstanceName(1); - - stopGrid(shutName, true, false); - } - else if( reqId != req.requestId() ){ - try { - U.sleep(IgniteSystemProperties.getLong(IGNITE_SQL_RETRY_TIMEOUT, GridReduceQueryExecutor.DFLT_RETRY_TIMEOUT)); - } - catch (IgniteInterruptedCheckedException e) { - // no-op - } - } - } - } - super.sendMessage(node, msg, ackC); - } - }); - cfg.setDiscoverySpi(new TcpDiscoverySpi(){ - public long getNodesJoined() { - return stats.joinedNodesCount(); - } - }); - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); - startGridsMultiThreaded(NODES_COUNT, false); - - personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") - .setBackups(1) - .setIndexedTypes(String.class, JoinSqlTestHelper.Person.class) - ); - final IgniteCache orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) - .setBackups(1) - .setIndexedTypes(String.class, JoinSqlTestHelper.Organization.class) - ); - - awaitPartitionMapExchange(); - - JoinSqlTestHelper.populateDataIntoOrg(orgCache); - JoinSqlTestHelper.populateDataIntoPerson(personCache); - } - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - } - - -} - diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java index ce385114485a7..1ead3aa5e5837 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java @@ -56,19 +56,14 @@ public class RetryCauseMessageSelfTest extends GridCommonAbstractTest { /** */ private static final int NODES_COUNT = 2; - /** */ private static final String ORG_SQL = "select * from Organization"; - /** */ private static final String ORG = "org"; - /** */ private IgniteCache personCache; - /** */ private IgniteCache orgCache; - /** */ private IgniteH2Indexing h2Idx; @@ -250,8 +245,7 @@ public void testPartitionedCacheReserveFailureMessage() { personCache.query(qry).getAll(); } catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED " + - "cache is not found or not in OWNING state) ")); + assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) [")); return; } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java deleted file mode 100644 index 815f9b0d06d38..0000000000000 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageTest.java +++ /dev/null @@ -1,444 +0,0 @@ -package org.apache.ignite.internal.processors.query.h2.twostep; - -import java.util.UUID; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReferenceArray; -import javax.cache.CacheException; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cache.CacheMode; -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.internal.GridKernalContext; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; -import org.apache.ignite.internal.processors.cache.distributed.dht.MockGridDhtLocalPartition; -import org.apache.ignite.internal.processors.query.GridQueryProcessor; -import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; -import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException; -import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; -import org.apache.ignite.internal.util.GridSpinBusyLock; -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.ipfinder.vm.TcpDiscoveryVmIpFinder; -import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; -import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; -import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; -import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.JOIN_SQL; -import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Organization; -import static org.apache.ignite.internal.processors.query.h2.twostep.JoinSqlTestHelper.Person; -/** - * Test for 6 retry cases - */ -public class RetryCauseMessageTest extends GridCommonAbstractTest { - /** IP finder. */ - private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); - - /** */ - private static final int NODES_COUNT = 2; - /** */ - private static final String ORG_SQL = "select * from Organization"; - /** */ - private static final String ORG = "org"; - /** */ - private IgniteCache personCache; - /** */ - private IgniteCache orgCache; - /** */ - private IgniteH2Indexing h2Idx; - - /** */ - @Override protected long getTestTimeout() { - return 600 * 1000; - } - - /** - * Failed to reserve partitions for query (cache is not found on local node) - */ - public void testSynthCacheWasNotFoundMessage() { - GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", - new MockGridMapQueryExecutor(null) { - @Override public void onMessage(UUID nodeId, Object msg) { - if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { - GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; - qryReq.caches().add(Integer.MAX_VALUE); - startedExecutor.onMessage(nodeId, msg); - qryReq.caches().remove(qryReq.caches().size() - 1); - } - else - startedExecutor.onMessage(nodeId, msg); - - } - }.insertRealExecutor(mapQryExec)); - - SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - } - catch (CacheException e) { - assertTrue(e.getMessage(), e.getMessage().contains("Failed to reserve partitions for query (cache is not found on local node) [")); - return; - } - finally { - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); - } - fail(); - } - - /** - * Failed to reserve partitions for query (group reservation failed) - */ - public void testGrpReservationFailureMessage() { - final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); - final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); - - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", - new MockGridMapQueryExecutor(null) { - @Override public void onMessage(UUID nodeId, Object msg) { - if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { - final MapReservationKey grpKey = new MapReservationKey(ORG, null); - reservations.put(grpKey, new GridReservable() { - @Override public boolean reserve() { - return false; - } - @Override public void release() {} - - }); - } - startedExecutor.onMessage(nodeId, msg); - - } - }.insertRealExecutor(mapQryExec)); - - SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - } - catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (group reservation failed) [")); - return; - } - finally { - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); - } - fail(); - } - - /** - * Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) - */ - public void testReplicatedCacheReserveFailureMessage() { - GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); - final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", - new MockGridMapQueryExecutor(null) { - @Override public void onMessage(UUID nodeId, Object msg) { - if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { - GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; - GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); - GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); - AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); - long stateVal = aState.getAndSet(2); - startedExecutor.onMessage(nodeId, msg); - aState.getAndSet(stateVal); - } - else - startedExecutor.onMessage(nodeId, msg); - } - }.insertRealExecutor(mapQryExec)); - - SqlQuery qry = new SqlQuery<>(Organization.class, ORG_SQL); - qry.setDistributedJoins(true); - try { - orgCache.query(qry).getAll(); - } - catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of REPLICATED cache is not in OWNING state) [")); - return; - } - finally { - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); - } - fail(); - } - - /** - * Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) - */ - public void testPartitionedCacheReserveFailureMessage() { - GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); - final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", - new MockGridMapQueryExecutor(null) { - @Override public void onMessage(UUID nodeId, Object msg) { - if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { - GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; - GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); - GridDhtLocalPartition part = cctx.topology().localPartition(0, NONE, false); - AtomicLong aState = GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "state"); - long stateVal = aState.getAndSet(2); - startedExecutor.onMessage(nodeId, msg); - aState.getAndSet(stateVal); - } - else - startedExecutor.onMessage(nodeId, msg); - - } - }.insertRealExecutor(mapQryExec)); - - SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - } - catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache cannot be reserved) [")); - return; - } - finally { - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); - } - fail(); - } - - /** - * Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) - */ - public void testPartitionStateChangedMessage() { - final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); - final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); - final GridKernalContext ctx = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "ctx"); - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", - new MockGridMapQueryExecutor(null) { - @Override public void onMessage(UUID nodeId, Object msg) { - if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { - GridH2QueryRequest qryReq = (GridH2QueryRequest)msg; - final MapReservationKey grpKey = new MapReservationKey(ORG, null); - reservations.put(grpKey, new GridReservable() { - @Override public boolean reserve() { - return true; - } - - @Override public void release() { - - } - }); - GridCacheContext cctx = ctx.cache().context().cacheContext(qryReq.caches().get(0)); - - if (GridDhtPartitionTopologyImpl.class.isAssignableFrom(cctx.topology().getClass())) { - GridDhtPartitionTopologyImpl tpg = (GridDhtPartitionTopologyImpl)(cctx.topology()); - AtomicReferenceArray locParts = GridTestUtils.getFieldValue(tpg, GridDhtPartitionTopologyImpl.class, "locParts"); - GridDhtLocalPartition part = locParts.get(0); - MockGridDhtLocalPartition mockPart = new MockGridDhtLocalPartition( - GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "ctx"), - GridTestUtils.getFieldValue(part, GridDhtLocalPartition.class, "grp"), - part) { - volatile private boolean preReserved = true; - - @Override public boolean reserve() { - preReserved = false; - return super.reserve(); - } - - @Override public GridDhtPartitionState state() { - if (preReserved) - return super.state(); - else - return MOVING; - } - }; - locParts.set(0, mockPart); - startedExecutor.onMessage(nodeId, msg); - locParts.set(0, part); - } - else - startedExecutor.onMessage(nodeId, msg); - - } - else - startedExecutor.onMessage(nodeId, msg); - - } - }.insertRealExecutor(mapQryExec)); - - SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - } - catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to reserve partitions for query (partition of PARTITIONED cache is not in OWNING state after reservation) [")); - return; - } - finally { - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); - } - fail(); - } - - /** - * Failed to execute non-collocated query (will retry) - */ - public void testNonCollocatedFailureMessage() { - final GridMapQueryExecutor mapQryExec = GridTestUtils.getFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec"); - final ConcurrentMap reservations = GridTestUtils.getFieldValue(mapQryExec, GridMapQueryExecutor.class, "reservations"); - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", - new MockGridMapQueryExecutor(null) { - @Override public void onMessage(UUID nodeId, Object msg) { - if (GridH2QueryRequest.class.isAssignableFrom(msg.getClass())) { - final MapReservationKey grpKey = new MapReservationKey(ORG, null); - reservations.put(grpKey, new GridReservable() { - @Override public boolean reserve() { - throw new GridH2RetryException("test retry exception"); - } - - @Override public void release() { - - } - }); - } - startedExecutor.onMessage(nodeId, msg); - - } - }.insertRealExecutor(mapQryExec)); - - SqlQuery qry = new SqlQuery(Person.class, JOIN_SQL).setArgs("Organization #0"); - qry.setDistributedJoins(true); - try { - personCache.query(qry).getAll(); - } - catch (CacheException e) { - assertTrue(e.getMessage().contains("Failed to execute non-collocated query (will retry) [")); - return; - } - finally { - GridTestUtils.setFieldValue(h2Idx, IgniteH2Indexing.class, "mapQryExec", mapQryExec); - } - fail(); - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setCommunicationSpi(new TcpCommunicationSpi(){ - /** {@inheritDoc} */ - @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { - assert msg != null; - - super.sendMessage(node, msg, ackC); - } - }); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - System.setProperty(IGNITE_SQL_RETRY_TIMEOUT, "5000"); - Ignite ignite = startGridsMultiThreaded(NODES_COUNT, false); - GridQueryProcessor qryProc = grid(ignite.name()).context().query(); - - h2Idx = GridTestUtils.getFieldValue(qryProc, GridQueryProcessor.class, "idx"); - - personCache = ignite(0).getOrCreateCache(new CacheConfiguration("pers") - .setIndexedTypes(String.class, Person.class) - ); - orgCache = ignite(0).getOrCreateCache(new CacheConfiguration(ORG) - .setCacheMode(CacheMode.REPLICATED) - .setIndexedTypes(String.class, Organization.class) - // .setNodeFilter( node -> node.order()<2L ) - ); - - awaitPartitionMapExchange(); - - JoinSqlTestHelper.populateDataIntoOrg(orgCache); - JoinSqlTestHelper.populateDataIntoPerson(personCache); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - } - - - /** - * Wrapper around @{GridMapQueryExecutor} - */ - private abstract static class MockGridMapQueryExecutor extends GridMapQueryExecutor { - - /** - * Wrapped executor - */ - GridMapQueryExecutor startedExecutor; - - /** */ - MockGridMapQueryExecutor insertRealExecutor(GridMapQueryExecutor realExecutor) { - this.startedExecutor = realExecutor; - return this; - } - - /** - * @param busyLock Busy lock. - */ - MockGridMapQueryExecutor(GridSpinBusyLock busyLock) { - super(busyLock); - } - - /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { - - startedExecutor.onMessage(nodeId, msg); - } - - /** {@inheritDoc} */ - @Override public void cancelLazyWorkers() { - - startedExecutor.cancelLazyWorkers(); - } - - /** {@inheritDoc} */ - @Override GridSpinBusyLock busyLock() { - - return startedExecutor.busyLock(); - - } - - /** {@inheritDoc} */ - @Override public void onCacheStop(String cacheName) { - - startedExecutor.onCacheStop(cacheName); - } - - /** {@inheritDoc} */ - @Override public void stopAndUnregisterCurrentLazyWorker() { - startedExecutor.stopAndUnregisterCurrentLazyWorker(); - } - - /** {@inheritDoc} */ - @Override public void unregisterLazyWorker(MapQueryLazyWorker worker) { - startedExecutor.unregisterLazyWorker(worker); - } - - /** {@inheritDoc} */ - @Override public int registeredLazyWorkers() { - - return startedExecutor.registeredLazyWorkers(); - } - } - -} From 59627ec2e89b5028252ace73e93eeb6d0b1f1536 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Wed, 29 Aug 2018 15:15:43 +0300 Subject: [PATCH 85/95] IGNITE-8913 Query cancelled messages are enriched with details, tests updated --- .../h2/twostep/GridReduceQueryExecutor.java | 109 +++++++++++------- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 92 ++++++++++----- 2 files changed, 128 insertions(+), 73 deletions(-) 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 1f2a36ab86f22..d79190ea41c28 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 @@ -802,24 +802,6 @@ public Iterator> query( boolean retry = false; - // Always enforce join order on map side to have consistent behavior. - int flags = GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER; - - if (distributedJoins) - flags |= GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS; - - if (qry.isLocal()) - flags |= GridH2QueryRequest.FLAG_IS_LOCAL; - - if (qry.explain()) - flags |= GridH2QueryRequest.FLAG_EXPLAIN; - - if (isReplicatedOnly) - flags |= GridH2QueryRequest.FLAG_REPLICATED; - - if (lazy && mapQrys.size() == 1) - flags |= GridH2QueryRequest.FLAG_LAZY; - GridH2QueryRequest req = new GridH2QueryRequest() .requestId(qryReqId) .topologyVersion(topVer) @@ -829,7 +811,7 @@ public Iterator> query( .partitions(convert(partsMap)) .queries(mapQrys) .parameters(params) - .flags(flags) + .flags(prepareFlags(qry, lazy, mapQrys.size())) .timeout(timeoutMillis) .schemaName(schemaName); @@ -875,28 +857,7 @@ else if (mvccTracker != null) if (send(nodes, req, spec, false)) { awaitAllReplies(r, nodes, cancel); - if (r.hasErrorOrRetry()) { - CacheException err = r.exception(); - - if (err != null) { - if (err.getCause() instanceof IgniteClientDisconnectedException) - throw err; - - if (wasCancelled(err)) - throw new QueryCancelledException(); // Throw correct exception. - - throw err; - } - else { - retry = true; - - // On-the-fly topology change must not be possible in FOR UPDATE case. - assert sfuFut == null; - - // If remote node asks us to retry then we have outdated full partition map. - h2.awaitForReadyTopologyVersion(r.retryTopologyVersion()); - } - } + retry = analyseCurrentRun(r); } else // Send failed. retry = true; @@ -992,6 +953,14 @@ else if (mvccTracker != null) Throwable disconnectedErr = ((IgniteCheckedException)e).getCause(IgniteClientDisconnectedException.class); + if ( QueryCancelledException.class.isAssignableFrom(e.getClass()) ) + cause = new QueryCancelledException(String.format( + "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s]", + qry.originalSql(), + ctx.localNodeId(), + "Cancelled by client" + )); + if (disconnectedErr != null) cause = disconnectedErr; } @@ -1016,6 +985,64 @@ else if (mvccTracker != null) } } + /** + * Analyse reduce query run to decide if retry is required + * @param r reduce query run to be analysed + * @return true if retry is required, false otherwise + * @throws IgniteCheckedException in case of reduce query run contains exception record + */ + private boolean analyseCurrentRun(ReduceQueryRun r) throws IgniteCheckedException { + if (r.hasError()) { + if (r.cacheException() != null) { + CacheException err = r.cacheException(); + + if (err.getCause() instanceof IgniteClientDisconnectedException) + throw err; + + Exception cause = wasCancelled(err) || X.hasCause(err, QueryCancelledException.class) + ? new QueryCancelledException(r.rootCause()) + : err; + + throw new CacheException("Failed to run map query remotely." + cause.getMessage(), cause); + } else { + // If remote node asks us to retry then we have outdated full partition map. + h2.awaitForReadyTopologyVersion(r.topVersion()); + + return true; + } + } + return false; + } + + /** + * Builds flag out of parameters + * @param qry query parameter holder + * @param lazy if lazy execution + * @param mapQrysSize number of queries + * @return flag + */ + private int prepareFlags(GridCacheTwoStepQuery qry, boolean lazy, int mapQrysSize) { + // Always enforce join order on map side to have consistent behavior. + int flags = GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER; + + if (qry.distributedJoins()) + flags |= GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS; + + if (qry.isLocal()) + flags |= GridH2QueryRequest.FLAG_IS_LOCAL; + + if (qry.explain()) + flags |= GridH2QueryRequest.FLAG_EXPLAIN; + + if (qry.isReplicatedOnly()) + flags |= GridH2QueryRequest.FLAG_REPLICATED; + + if (lazy && mapQrysSize == 1) + flags |= GridH2QueryRequest.FLAG_LAZY; + + return flags; + } + /** * * @param schemaName Schema name. diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index a0cea727cc6b0..d05a33e3c39c9 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -35,6 +35,7 @@ import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +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.spi.discovery.tcp.TcpDiscoverySpi; @@ -46,7 +47,7 @@ */ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends GridCommonAbstractTest { /** Grids count. */ - private static final int GRIDS_CNT = 3; + private static final int GRIDS_CNT = 4; /** IP finder. */ private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -66,6 +67,12 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr /** */ private static final String QRY_3 = "select a._val from String a"; + /** */ + private static final String CANCELLED_BY_CLIENT = "reason=Cancelled by client"; + + /** */ + private static final String WITH_TIMEOUT_WAS_CANCELLED = "reason=Statement with timeout was cancelled"; + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); @@ -100,82 +107,96 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr /** */ public void testRemoteQueryExecutionTimeout() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, true); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, true, + WITH_TIMEOUT_WAS_CANCELLED); } /** */ - public void testRemoteQueryWithMergeTableTimeout() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, true); + public void testRemoteQueryWithMergeTableTimeout0() throws Exception { + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.MILLISECONDS, true, + WITH_TIMEOUT_WAS_CANCELLED); + } + + /** Query possibly could be executed faster than timeout*/ + public void testRemoteQueryWithMergeTableTimeout1() throws Exception { + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 25, TimeUnit.MILLISECONDS, true, + WITH_TIMEOUT_WAS_CANCELLED); } /** */ public void testRemoteQueryExecutionCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryExecutionCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryExecutionCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryExecutionCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false, + CANCELLED_BY_CLIENT); } /** */ public void testRemoteQueryWithMergeTableCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithMergeTableCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithMergeTableCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithMergeTableCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false, null); } - /** */ + /** Query possibly could be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(2*CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false, + CANCELLED_BY_CLIENT); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); } - /** */ + /** Query with far less complex sql and expected to be executed faster than timeout*/ public void testRemoteQueryAlreadyFinishedStop() throws Exception { - testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); + testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); } /** */ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, - boolean timeout) throws Exception { + boolean timeout, String cause) throws Exception { try (Ignite client = startGrid("client")) { IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); @@ -216,18 +237,25 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni try(QueryCursor> ignored = cursor) { cursor.iterator(); - fail(); + + if (!F.isEmpty(cause)) + fail("No exception caught"); } catch (CacheException ex) { - log().error("Got expected exception", ex); + log().error("Got exception", ex); + + log().error( "Cause of exception", ex.getCause()); assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException); - } - // Give some time to clean up. - Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); + assertTrue( "Cause message "+ex.getCause().getMessage(), ex.getCause().getMessage().contains(cause)); + }finally { - checkCleanState(); + // Give some time to clean up. + Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); + + checkCleanState(); + } } } From ee8203907797e97dea8ad25e0d4dd778afe04dbf Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 11 Sep 2018 14:31:13 +0300 Subject: [PATCH 86/95] IGNITE-8913 rebase onto master --- .../h2/twostep/GridReduceQueryExecutor.java | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) 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 d79190ea41c28..3ea6572129026 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 @@ -857,7 +857,7 @@ else if (mvccTracker != null) if (send(nodes, req, spec, false)) { awaitAllReplies(r, nodes, cancel); - retry = analyseCurrentRun(r); + retry = analyseCurrentRun(r, sfuFut); } else // Send failed. retry = true; @@ -988,25 +988,30 @@ else if (mvccTracker != null) /** * Analyse reduce query run to decide if retry is required * @param r reduce query run to be analysed + * @param sfuFut * @return true if retry is required, false otherwise * @throws IgniteCheckedException in case of reduce query run contains exception record */ - private boolean analyseCurrentRun(ReduceQueryRun r) throws IgniteCheckedException { - if (r.hasError()) { - if (r.cacheException() != null) { - CacheException err = r.cacheException(); + private boolean analyseCurrentRun(ReduceQueryRun r, GridNearTxSelectForUpdateFuture sfuFut) + throws IgniteCheckedException { + if (r.hasErrorOrRetry()) { + if (r.exception() != null) { + CacheException err = r.exception(); if (err.getCause() instanceof IgniteClientDisconnectedException) throw err; Exception cause = wasCancelled(err) || X.hasCause(err, QueryCancelledException.class) - ? new QueryCancelledException(r.rootCause()) + ? new QueryCancelledException(r.retryCause()) : err; throw new CacheException("Failed to run map query remotely." + cause.getMessage(), cause); } else { + // On-the-fly topology change must not be possible in FOR UPDATE case. + assert sfuFut == null; + // If remote node asks us to retry then we have outdated full partition map. - h2.awaitForReadyTopologyVersion(r.topVersion()); + h2.awaitForReadyTopologyVersion(r.retryTopologyVersion()); return true; } From 3284a77b76917ff5469ffe156c6403df8d07307b Mon Sep 17 00:00:00 2001 From: Yury Gerzhedovich Date: Wed, 12 Sep 2018 11:35:37 +0300 Subject: [PATCH 87/95] ignite-8913: minor fixes --- .../processors/query/h2/IgniteH2Indexing.java | 4 +- .../h2/twostep/GridMapQueryExecutor.java | 2 +- .../h2/twostep/GridReduceQueryExecutor.java | 38 ++++++++++--------- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 19 +++++----- 4 files changed, 34 insertions(+), 29 deletions(-) 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 f94430a6f3ade..745b32da09903 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 @@ -128,6 +128,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable; import org.apache.ignite.internal.processors.query.h2.sys.SqlSystemTableEngine; +import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemView; import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewBaselineNodes; import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeAttributes; import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeMetrics; @@ -136,7 +137,6 @@ import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor; import org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker; import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest; -import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemView; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl; @@ -1404,7 +1404,7 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", stmt, ctx.localNodeId(), - timeoutMillis>0 ? "Statement with timeout was cancelled" : "Cancelled by client", + timeoutMillis > 0 ? "Statement with timeout was cancelled" : "Cancelled by client", timeoutMillis ))); 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 baa5f9c074dda..895c4769c1eb2 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 @@ -1053,7 +1053,7 @@ private void onQueryRequest0( else { U.error(log, "Failed to execute local query.", e); - Exception cancelled = X.cause(e,QueryCancelledException.class); + Exception cancelled = X.cause(e, QueryCancelledException.class); sendError(node, reqId, (cancelled != null) ? cancelled : e); 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 0c454c3313750..2c281600f12bc 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 @@ -39,7 +39,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import javax.cache.CacheException; @@ -114,7 +113,9 @@ import static java.util.Collections.singletonList; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_RETRY_TIMEOUT; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; -import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.*; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.checkActive; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccEnabled; +import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.tx; import static org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery.EMPTY_PARAMS; import static org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode.OFF; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.REDUCE; @@ -623,7 +624,7 @@ public Iterator> query( if (qry.forUpdate()) { // Indexing should have started TX at this point for FOR UPDATE query. assert mvccEnabled && curTx != null; - + try { TxTopologyVersionFuture topFut = new TxTopologyVersionFuture(curTx, mvccTracker.context()); @@ -800,8 +801,6 @@ public Iterator> query( } }); - boolean retry = false; - GridH2QueryRequest req = new GridH2QueryRequest() .requestId(qryReqId) .topologyVersion(topVer) @@ -854,6 +853,8 @@ else if (mvccTracker != null) else spec = pspec; + boolean retry; + if (send(nodes, req, spec, false)) { awaitAllReplies(r, nodes, cancel); @@ -953,7 +954,7 @@ else if (mvccTracker != null) Throwable disconnectedErr = ((IgniteCheckedException)e).getCause(IgniteClientDisconnectedException.class); - if ( QueryCancelledException.class.isAssignableFrom(e.getClass()) ) + if (QueryCancelledException.class.isAssignableFrom(e.getClass())) cause = new QueryCancelledException(String.format( "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s]", qry.originalSql(), @@ -986,11 +987,12 @@ else if (mvccTracker != null) } /** - * Analyse reduce query run to decide if retry is required - * @param r reduce query run to be analysed - * @param sfuFut - * @return true if retry is required, false otherwise - * @throws IgniteCheckedException in case of reduce query run contains exception record + * Analyse reduce query run to decide if retry is required. + * + * @param r Reduce query run to be analysed. + * @param sfuFut Grid near tx select for update future. + * @return {@code true} if retry is required, {@code false} otherwise. + * @throws IgniteCheckedException In case of reduce query run contains exception record. */ private boolean analyseCurrentRun(ReduceQueryRun r, GridNearTxSelectForUpdateFuture sfuFut) throws IgniteCheckedException { @@ -1006,7 +1008,8 @@ private boolean analyseCurrentRun(ReduceQueryRun r, GridNearTxSelectForUpdateFut : err; throw new CacheException("Failed to run map query remotely." + cause.getMessage(), cause); - } else { + } + else { // On-the-fly topology change must not be possible in FOR UPDATE case. assert sfuFut == null; @@ -1020,11 +1023,12 @@ private boolean analyseCurrentRun(ReduceQueryRun r, GridNearTxSelectForUpdateFut } /** - * Builds flag out of parameters - * @param qry query parameter holder - * @param lazy if lazy execution - * @param mapQrysSize number of queries - * @return flag + * Builds flag out of parameters. + * + * @param qry Query parameter holder. + * @param lazy If lazy execution. + * @param mapQrysSize Number of queries. + * @return flag. */ private int prepareFlags(GridCacheTwoStepQuery qry, boolean lazy, int mapQrysSize) { // Always enforce join order on map side to have consistent behavior. diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index d05a33e3c39c9..490cb4425d37e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -27,12 +27,12 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.util.typedef.F; @@ -196,7 +196,7 @@ public void testRemoteQueryAlreadyFinishedStop() throws Exception { /** */ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, - boolean timeout, String cause) throws Exception { + boolean timeout, String cause) throws Exception { try (Ignite client = startGrid("client")) { IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); @@ -209,7 +209,7 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni Arrays.fill(tmp, ' '); cache.put(i, new String(tmp)); - if (i/(float)keyCnt >= p/10f) { + if (i / (float)keyCnt >= p / 10f) { log().info("Loaded " + i + " of " + keyCnt); p++; @@ -225,7 +225,8 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni qry.setTimeout(timeoutUnits, timeUnit); cursor = cache.query(qry); - } else { + } + else { cursor = cache.query(qry); client.scheduler().runLocal(new Runnable() { @@ -235,7 +236,7 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni }, timeoutUnits, timeUnit); } - try(QueryCursor> ignored = cursor) { + try (QueryCursor> ignored = cursor) { cursor.iterator(); if (!F.isEmpty(cause)) @@ -244,13 +245,13 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni catch (CacheException ex) { log().error("Got exception", ex); - log().error( "Cause of exception", ex.getCause()); + log().error("Cause of exception", ex.getCause()); assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException); - assertTrue( "Cause message "+ex.getCause().getMessage(), ex.getCause().getMessage().contains(cause)); - }finally { - + assertTrue("Cause message " + ex.getCause().getMessage(), ex.getCause().getMessage().contains(cause)); + } + finally { // Give some time to clean up. Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); From f510922a4f16e11170195d2cc5f6b873f4763fe9 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Wed, 12 Sep 2018 12:32:53 +0300 Subject: [PATCH 88/95] IGNITE-8913 fixes according to review --- .../cache/query/QueryCancelledException.java | 5 +- .../processors/cache/QueryCursorImpl.java | 7 +- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 85 +++++++++++++++---- 3 files changed, 75 insertions(+), 22 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java index 3e0662f08e9cb..6ef21c8faa1da 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java @@ -33,7 +33,10 @@ public QueryCancelledException() { super("The qry was cancelled while executing"); } - /** */ + /** + * Customized constructor receives message + * @param msg Message + */ public QueryCancelledException(String msg) { super(msg); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java index 9cb0a130f9d20..382f3aef900b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java @@ -98,10 +98,9 @@ public QueryCursorImpl(Iterable iterExec, GridQueryCancel cancel, boolean isQ // Handle race with cancel and make sure the iterator resources are freed correctly. closeIter(); - throw new CacheException(new QueryCancelledException(String.format( - "The query was cancelled while executing. Client node should provide details [reason=%s]", - "Cancelled by client" - ))); + throw new CacheException(new QueryCancelledException("The query was cancelled while executing. " + + "Client node should provide details [reason=Cancelled by client]" + )); } assert iter != null; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 490cb4425d37e..7e9a8d93fa834 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -105,96 +105,147 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr g.cache(DEFAULT_CACHE_NAME).removeAll(); } - /** */ + /** + * @throws Exception If failed. + */ public void testRemoteQueryExecutionTimeout() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, true, WITH_TIMEOUT_WAS_CANCELLED); } - /** */ + /** + * @throws Exception If failed. + */ public void testRemoteQueryWithMergeTableTimeout0() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.MILLISECONDS, true, WITH_TIMEOUT_WAS_CANCELLED); } - /** Query possibly could be executed faster than timeout*/ + /** + * Query possibly could be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithMergeTableTimeout1() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 25, TimeUnit.MILLISECONDS, true, WITH_TIMEOUT_WAS_CANCELLED); } - /** */ + /** + * @throws Exception If failed. + */ public void testRemoteQueryExecutionCancel0() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false, CANCELLED_BY_CLIENT); } - /** */ + /** + * @throws Exception If failed. + */ public void testRemoteQueryExecutionCancel1() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, false, CANCELLED_BY_CLIENT); } - /** */ + /** + * @throws Exception If failed. + */ public void testRemoteQueryExecutionCancel2() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false, CANCELLED_BY_CLIENT); } - /** */ + /** + * @throws Exception If failed. + */ public void testRemoteQueryExecutionCancel3() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false, CANCELLED_BY_CLIENT); } - /** */ + /** + * @throws Exception If failed. + */ public void testRemoteQueryWithMergeTableCancel0() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false, CANCELLED_BY_CLIENT); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** + * Query with far less complex sql and expected to be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithMergeTableCancel1() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false, null); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** + * Query with far less complex sql and expected to be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithMergeTableCancel2() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false, null); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** + * Query with far less complex sql and expected to be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithMergeTableCancel3() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false, null); } - /** Query possibly could be executed faster than timeout*/ + /** + * Query possibly could be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { testQueryCancel(2*CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false, CANCELLED_BY_CLIENT); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** + * Query with far less complex sql and expected to be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false, null); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** + * Query with far less complex sql and expected to be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false, null); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** + * Query with far less complex sql and expected to be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); } - /** Query with far less complex sql and expected to be executed faster than timeout*/ + /** + * Query with far less complex sql and expected to be executed faster than timeout + * @throws Exception If failed. + */ public void testRemoteQueryAlreadyFinishedStop() throws Exception { testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false, null); } - /** */ + /** + * Common logic for tests + * @param keyCnt Cash size + * @param valSize Length of value + * @param sql Select query + * @param timeoutUnits Timeout + * @param timeUnit Measurement units + * @param timeout Is timeout set + * @param cause Expected cause of exception + * @throws Exception Uncaught exception + */ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, boolean timeout, String cause) throws Exception { try (Ignite client = startGrid("client")) { From 70f330a4889dffb4460da59e60c61bedb555bafa Mon Sep 17 00:00:00 2001 From: Yury Gerzhedovich Date: Wed, 12 Sep 2018 13:10:27 +0300 Subject: [PATCH 89/95] ignite-8913: minor fixes --- .../cache/query/QueryCancelledException.java | 9 ++-- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 44 ++++++++++++------- 2 files changed, 30 insertions(+), 23 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java index 6ef21c8faa1da..3121a12cf26fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java @@ -20,7 +20,7 @@ import org.apache.ignite.IgniteCheckedException; /** - * The exception is thrown if a qry was cancelled or timed out while executing. + * The exception is thrown if a query was cancelled or timed out while executing. */ public class QueryCancelledException extends IgniteCheckedException { /** */ @@ -30,13 +30,10 @@ public class QueryCancelledException extends IgniteCheckedException { * Default constructor. */ public QueryCancelledException() { - super("The qry was cancelled while executing"); + super("The query was cancelled while executing"); } - /** - * Customized constructor receives message - * @param msg Message - */ + /** {@inheritDoc} */ public QueryCancelledException(String msg) { super(msg); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 7e9a8d93fa834..57f090e9fb3f2 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -122,7 +122,8 @@ public void testRemoteQueryWithMergeTableTimeout0() throws Exception { } /** - * Query possibly could be executed faster than timeout + * Query possibly could be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryWithMergeTableTimeout1() throws Exception { @@ -172,6 +173,7 @@ public void testRemoteQueryWithMergeTableCancel0() throws Exception { /** * Query with far less complex sql and expected to be executed faster than timeout + * * @throws Exception If failed. */ public void testRemoteQueryWithMergeTableCancel1() throws Exception { @@ -179,7 +181,8 @@ public void testRemoteQueryWithMergeTableCancel1() throws Exception { } /** - * Query with far less complex sql and expected to be executed faster than timeout + * Query with far less complex sql and expected to be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryWithMergeTableCancel2() throws Exception { @@ -187,7 +190,8 @@ public void testRemoteQueryWithMergeTableCancel2() throws Exception { } /** - * Query with far less complex sql and expected to be executed faster than timeout + * Query with far less complex sql and expected to be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryWithMergeTableCancel3() throws Exception { @@ -195,7 +199,8 @@ public void testRemoteQueryWithMergeTableCancel3() throws Exception { } /** - * Query possibly could be executed faster than timeout + * Query possibly could be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { @@ -204,7 +209,8 @@ public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { } /** - * Query with far less complex sql and expected to be executed faster than timeout + * Query with far less complex sql and expected to be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { @@ -212,7 +218,8 @@ public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { } /** - * Query with far less complex sql and expected to be executed faster than timeout + * Query with far less complex sql and expected to be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { @@ -220,7 +227,8 @@ public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { } /** - * Query with far less complex sql and expected to be executed faster than timeout + * Query with far less complex sql and expected to be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { @@ -228,7 +236,8 @@ public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { } /** - * Query with far less complex sql and expected to be executed faster than timeout + * Query with far less complex sql and expected to be executed faster than timeout. + * * @throws Exception If failed. */ public void testRemoteQueryAlreadyFinishedStop() throws Exception { @@ -236,15 +245,16 @@ public void testRemoteQueryAlreadyFinishedStop() throws Exception { } /** - * Common logic for tests - * @param keyCnt Cash size - * @param valSize Length of value - * @param sql Select query - * @param timeoutUnits Timeout - * @param timeUnit Measurement units - * @param timeout Is timeout set - * @param cause Expected cause of exception - * @throws Exception Uncaught exception + * Common logic for tests. + * + * @param keyCnt Cash size. + * @param valSize Length of value. + * @param sql Select query. + * @param timeoutUnits Timeout. + * @param timeUnit Measurement units. + * @param timeout Is timeout set. + * @param cause Expected cause of exception. + * @throws Exception Uncaught exception. */ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, boolean timeout, String cause) throws Exception { From 6b093cc20f0a6d2d30c1f8585e9b22149eaf018b Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Thu, 13 Sep 2018 16:59:02 +0300 Subject: [PATCH 90/95] IGNITE-8913 stack trace optimization --- .../org/apache/ignite/cache/query/QueryCancelledException.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java index 3121a12cf26fd..b81df69586309 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java @@ -35,6 +35,6 @@ public QueryCancelledException() { /** {@inheritDoc} */ public QueryCancelledException(String msg) { - super(msg); + super(msg, null, false); } } From 55ca7ac6c768e27222c81db00658340e98c1b510 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Tue, 18 Sep 2018 17:44:45 +0300 Subject: [PATCH 91/95] IGNITE-8913 fixed according to review --- .../processors/cache/QueryCursorImpl.java | 4 +-- .../processors/query/h2/IgniteH2Indexing.java | 36 ++++++++++++------- .../h2/twostep/GridMapQueryExecutor.java | 6 ++-- .../h2/twostep/GridReduceQueryExecutor.java | 15 ++++---- ...acheLocalQueryCancelOrTimeoutSelfTest.java | 2 +- 5 files changed, 35 insertions(+), 28 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java index 382f3aef900b6..b4587571f98e0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java @@ -98,9 +98,7 @@ public QueryCursorImpl(Iterable iterExec, GridQueryCancel cancel, boolean isQ // Handle race with cancel and make sure the iterator resources are freed correctly. closeIter(); - throw new CacheException(new QueryCancelledException("The query was cancelled while executing. " + - "Client node should provide details [reason=Cancelled by client]" - )); + throw new CacheException(new QueryCancelledException("Query cursor was cancelled explicitly")); } assert iter != null; 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 745b32da09903..5462684d994c5 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 @@ -1371,19 +1371,29 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement final MapQueryLazyWorker lazyWorker = MapQueryLazyWorker.currentWorker(); if (cancel != null) { - cancel.set(new Runnable() { - @Override public void run() { - if (lazyWorker != null) { - lazyWorker.submit(new Runnable() { - @Override public void run() { - cancelStatement(stmt); - } - }); + try { + cancel.set(new Runnable() { + @Override public void run() { + if (lazyWorker != null) { + lazyWorker.submit(new Runnable() { + @Override public void run() { + cancelStatement(stmt); + } + }); + } + else + cancelStatement(stmt); } - else - cancelStatement(stmt); - } - }); + }); + }catch(QueryCancelledException e){ + throw new CacheException(new QueryCancelledException(String.format( + "The query was cancelled before executing [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", + stmt, + ctx.localNodeId(), + timeoutMillis > 0 ? "Statement with timeout was cancelled" : "Cancelled by client", + timeoutMillis + ))); + } } Session ses = H2Utils.session(conn); @@ -1401,7 +1411,7 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement // Throw special exception. if (e.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED) throw new CacheException(new QueryCancelledException(String.format( - "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", + "The query was cancelled while executing [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", stmt, ctx.localNodeId(), timeoutMillis > 0 ? "Statement with timeout was cancelled" : "Cancelled by client", 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 895c4769c1eb2..af1b6096e5c4a 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 @@ -891,7 +891,7 @@ private void onQueryRequest0( nodeRess.cancelRequest(reqId); throw new QueryCancelledException(String.format( - "The query request (could be more than 1 query) was cancelled while executing. " + + "The query request was cancelled while executing " + "[reqId=%s, firstQuery=%s, localNodeId=%s, reason=%s]", reqId, qrys.isEmpty() ? "no queries" : qrys.iterator().next().query(), @@ -1113,7 +1113,7 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th String err = reservePartitions(cacheIds, topVer, parts, reserved, node.id(), reqId); if (!F.isEmpty(err)) { - U.error(log, "Failed to reserve partitions for DML request. [localNodeId=" + ctx.localNodeId() + + U.error(log, "Failed to reserve partitions for DML request [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", reqId=" + req.requestId() + ", cacheIds=" + cacheIds + ", topVer=" + topVer + ", parts=" + Arrays.toString(parts) + ']'); @@ -1174,7 +1174,7 @@ private void onDmlRequest(final ClusterNode node, final GridH2DmlRequest req) th sendUpdateResponse(node, reqId, updRes, null); } catch (Exception e) { - U.error(log, "Error processing dml request. [localNodeId=" + ctx.localNodeId() + + U.error(log, "Error processing dml request [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", req=" + req + ']', e); sendUpdateResponse(node, reqId, null, e.getMessage()); 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 2c281600f12bc..f29db320c9b2a 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 @@ -956,7 +956,7 @@ else if (mvccTracker != null) if (QueryCancelledException.class.isAssignableFrom(e.getClass())) cause = new QueryCancelledException(String.format( - "The query was cancelled while executing. [query=%s, localNodeId=%s, reason=%s]", + "The query was cancelled while executing [query=%s, localNodeId=%s, reason=%s]", qry.originalSql(), ctx.localNodeId(), "Cancelled by client" @@ -1003,11 +1003,10 @@ private boolean analyseCurrentRun(ReduceQueryRun r, GridNearTxSelectForUpdateFut if (err.getCause() instanceof IgniteClientDisconnectedException) throw err; - Exception cause = wasCancelled(err) || X.hasCause(err, QueryCancelledException.class) - ? new QueryCancelledException(r.retryCause()) - : err; + if( wasCancelled(err) || X.hasCause(err, QueryCancelledException.class)) + throw new QueryCancelledException(r.retryCause()); - throw new CacheException("Failed to run map query remotely." + cause.getMessage(), cause); + throw err; } else { // On-the-fly topology change must not be possible in FOR UPDATE case. @@ -1105,7 +1104,7 @@ public UpdateResult update( for (ClusterNode n : nodes) { if (!n.version().greaterThanEqual(2, 3, 0)) { log.warning("Server-side DML optimization is skipped because map node does not support it. " + - "Falling back to normal DML. [node=" + n.id() + ", v=" + n.version() + "]."); + "Falling back to normal DML [node=" + n.id() + ", v=" + n.version() + "]."); return null; } @@ -1185,7 +1184,7 @@ private void onDmlResponse(final ClusterNode node, GridH2DmlResponse msg) { DistributedUpdateRun r = updRuns.get(reqId); if (r == null) { - U.warn(log, "Unexpected dml response (will ignore). [localNodeId=" + ctx.localNodeId() + ", nodeId=" + + U.warn(log, "Unexpected dml response (will ignore) [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", msg=" + msg.toString() + ']'); return; @@ -1194,7 +1193,7 @@ private void onDmlResponse(final ClusterNode node, GridH2DmlResponse msg) { r.handleResponse(node.id(), msg); } catch (Exception e) { - U.error(log, "Error in dml response processing. [localNodeId=" + ctx.localNodeId() + ", nodeId=" + + U.error(log, "Error in dml response processing [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", msg=" + msg.toString() + ']', e); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java index 4e74ac1e60489..759e55b6a3ba4 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java @@ -106,7 +106,7 @@ public void testQueryCancel() { * Tests cancellation with zero timeout. */ public void testQueryCancelZeroTimeout() { - testQuery(false, 1, TimeUnit.MILLISECONDS, CANCELLED_BY_CLIENT); + testQuery(false, 10, TimeUnit.NANOSECONDS, CANCELLED_BY_CLIENT); } /** From b84edd5453637e67f1cc57092f91d50670d3c3b7 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Wed, 26 Sep 2018 19:32:24 +0300 Subject: [PATCH 92/95] IGNITE-8913 adjusted according to review --- .../processors/query/GridQueryCancel.java | 46 +++++++++++++++---- .../processors/query/h2/IgniteH2Indexing.java | 20 ++++---- .../h2/twostep/GridReduceQueryExecutor.java | 32 +++++++++---- 3 files changed, 70 insertions(+), 28 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java index d49bcc7ae77f0..ee70d5365d60b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java @@ -25,32 +25,40 @@ */ public class GridQueryCancel { /** No-op runnable indicating cancelled state. */ - private static final Runnable CANCELLED = new Runnable() { + private static final Cancellable CANCELLED = new Cancellable() { + + @Override public String buildExceptionMessage() { + return null; //checked + } + @Override public void run() { // No-op. } }; /** */ - private static final AtomicReferenceFieldUpdater STATE_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(GridQueryCancel.class, Runnable.class, "clo"); + private static final AtomicReferenceFieldUpdater STATE_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(GridQueryCancel.class, Cancellable.class, "clo"); /** */ - private volatile Runnable clo; + private volatile Cancellable clo; + + /** */ + private volatile String msg; /** * Sets a cancel closure. * * @param clo Clo. */ - public void set(Runnable clo) throws QueryCancelledException { + public void set(Cancellable clo) throws QueryCancelledException { assert clo != null; while(true) { - Runnable tmp = this.clo; + Cancellable tmp = this.clo; if (tmp == CANCELLED) - throw new QueryCancelledException(); + throw new QueryCancelledException(msg != null ? msg : clo.buildExceptionMessage()); if (STATE_UPDATER.compareAndSet(this, tmp, clo)) return; @@ -62,7 +70,15 @@ public void set(Runnable clo) throws QueryCancelledException { */ public void cancel() { while(true) { - Runnable tmp = this.clo; + Cancellable tmp = this.clo; + + if (tmp != null) { + String msg = tmp.buildExceptionMessage(); + + //CANCELLED state has null + if (msg != null) + this.msg = msg; + } if (STATE_UPDATER.compareAndSet(this, tmp, CANCELLED)) { if (tmp != null) @@ -78,6 +94,16 @@ public void cancel() { */ public void checkCancelled() throws QueryCancelledException { if (clo == CANCELLED) - throw new QueryCancelledException(); + throw new QueryCancelledException(msg != null ? msg : "The query was cancelled before initialization."); + } + + /** + * Special interface for closure + */ + public interface Cancellable extends Runnable { + /** + * @return Message to show in case of exception + */ + String buildExceptionMessage(); } -} \ No newline at end of file +} 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 6664abbc640b4..905ac4d9124ea 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 @@ -1391,7 +1391,7 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement if (cancel != null) { try { - cancel.set(new Runnable() { + cancel.set(new GridQueryCancel.Cancellable() { @Override public void run() { if (lazyWorker != null) { lazyWorker.submit(new Runnable() { @@ -1403,15 +1403,19 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement else cancelStatement(stmt); } + + @Override public String buildExceptionMessage(){ + return String.format( + "The query was cancelled before executing [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", + stmt, + nodeId, + timeoutMillis > 0 ? "Statement with timeout was cancelled" : "Cancelled by client", + timeoutMillis + ); + } }); }catch(QueryCancelledException e){ - throw new CacheException(new QueryCancelledException(String.format( - "The query was cancelled before executing [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", - stmt, - ctx.localNodeId(), - timeoutMillis > 0 ? "Statement with timeout was cancelled" : "Cancelled by client", - timeoutMillis - ))); + throw new CacheException(e); } } 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 f29db320c9b2a..8a42c4819d51c 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 @@ -795,10 +795,20 @@ public Iterator> query( final long qryReqId0 = qryReqId; - cancel.set(new Runnable() { + cancel.set(new GridQueryCancel.Cancellable() { @Override public void run() { send(finalNodes, new GridQueryCancelRequest(qryReqId0), null, false); } + + @Override public String buildExceptionMessage(){ + return String.format( + "The query was cancelled while executing [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", + qry.originalSql(), + ctx.localNodeId(), + timeoutMillis > 0 ? "Statement with timeout was cancelled" : "Cancelled by client", + timeoutMillis + ); + } }); GridH2QueryRequest req = new GridH2QueryRequest() @@ -954,14 +964,6 @@ else if (mvccTracker != null) Throwable disconnectedErr = ((IgniteCheckedException)e).getCause(IgniteClientDisconnectedException.class); - if (QueryCancelledException.class.isAssignableFrom(e.getClass())) - cause = new QueryCancelledException(String.format( - "The query was cancelled while executing [query=%s, localNodeId=%s, reason=%s]", - qry.originalSql(), - ctx.localNodeId(), - "Cancelled by client" - )); - if (disconnectedErr != null) cause = disconnectedErr; } @@ -1141,12 +1143,22 @@ public UpdateResult update( final Collection finalNodes = nodes; - cancel.set(new Runnable() { + cancel.set(new GridQueryCancel.Cancellable() { @Override public void run() { r.future().onCancelled(); send(finalNodes, new GridQueryCancelRequest(reqId), null, false); } + + @Override public String buildExceptionMessage(){ + return String.format( + "The query was cancelled [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", + selectQry, + ctx.localNodeId(), + timeoutMillis > 0 ? "Query with timeout was cancelled" : "Cancelled by client", + timeoutMillis + ); + } }); // send() logs the debug message From fffcf4b0579f662efa16b78bf20570cffea583a3 Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Wed, 17 Oct 2018 14:19:08 +0300 Subject: [PATCH 93/95] IGNITE-8913 after merge fix --- .../ignite/internal/processors/query/h2/IgniteH2Indexing.java | 2 -- .../processors/query/h2/twostep/GridReduceQueryExecutor.java | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) 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 8e037fd299f94..2008089bfbcef 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 @@ -1449,8 +1449,6 @@ public PreparedStatement preparedStatementWithParams(Connection conn, String sql */ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement stmt, int timeoutMillis, @Nullable GridQueryCancel cancel) throws IgniteCheckedException { - final MapQueryLazyWorker lazyWorker = MapQueryLazyWorker.currentWorker(); - if (cancel != null) { try { cancel.set(new GridQueryCancel.Cancellable() { 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 7806009c848c9..ecf0343d8bae2 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 @@ -1011,7 +1011,7 @@ private boolean analyseCurrentRun(ReduceQueryRun r, GridNearTxSelectForUpdateFut throw err; if( wasCancelled(err) || X.hasCause(err, QueryCancelledException.class)) - throw new QueryCancelledException(r.retryCause()); + throw new QueryCancelledException(err.getMessage()); throw err; } From 11c7bb3815834519790e800321eb44d1c2567b0c Mon Sep 17 00:00:00 2001 From: SGrimstad Date: Fri, 19 Oct 2018 15:24:33 +0300 Subject: [PATCH 94/95] IGNITE-8913 after merge fix --- .../h2/twostep/GridMapQueryExecutor.java | 50 +++++++++++++++++-- .../h2/twostep/GridReduceQueryExecutor.java | 41 +++++++++++---- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 13 +++-- 3 files changed, 86 insertions(+), 18 deletions(-) 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 99daeb9d99504..e3a006c5c0006 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 @@ -792,6 +792,8 @@ private void onQueryRequest0( List reserved = new ArrayList<>(); + GridCacheSqlQuery lastQry = null; + try { // We want to reserve only in not SELECT FOR UPDATE case - // otherwise, their state is protected by locked topology. @@ -862,6 +864,8 @@ private void onQueryRequest0( boolean evt = mainCctx != null && mainCctx.events().isRecordable(EVT_CACHE_QUERY_EXECUTED); for (GridCacheSqlQuery qry : qrys) { + lastQry = qry; + ResultSet rs = null; boolean removeMapping = false; @@ -941,7 +945,14 @@ private void onQueryRequest0( if (qryResults.cancelled()) { qryResults.result(qryIdx).close(); - throw new QueryCancelledException(); + throw new QueryCancelledException( + String.format("The query was cancelled while executing [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", + qry.query(), + node.id(), + timeout > 0 ? "Query with timeout was cancelled" : "Cancelled by client", + timeout + ) + ); } if (inTx) { @@ -1011,7 +1022,15 @@ else if (!qryResults.isAllClosed()) { JdbcSQLException sqlEx = X.cause(e, JdbcSQLException.class); if (sqlEx != null && sqlEx.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED) - sendError(node, reqId, new QueryCancelledException()); + sendError(node, reqId, new QueryCancelledException( + String.format( + "The query was cancelled while executing [query=%s, nodeId=%s, reason=%s, timeout=%s ms]", + (lastQry != null) ? lastQry.query() : "no query", + node.id(), + timeout > 0 ? "Statement with timeout was cancelled" : "Cancelled by client", + timeout + ) + )); else { GridH2RetryException retryErr = X.cause(e, GridH2RetryException.class); @@ -1248,7 +1267,14 @@ private void onNextPageRequest(final ClusterNode node, final GridQueryNextPageRe return; } else if (nodeRess.cancelled(req.queryRequestId())) { - sendError(node, req.queryRequestId(), new QueryCancelledException()); + sendError(node, req.queryRequestId(), new QueryCancelledException( + String.format("The query was cancelled while executing [queryId=%d, nodeId=%s, requestId=%s, reason=%s]", + req.query(), + node.id(), + req.queryRequestId(), + "Cancellation in node result" + ) + )); return; } @@ -1258,7 +1284,14 @@ else if (nodeRess.cancelled(req.queryRequestId())) { if (qryResults == null) sendError(node, req.queryRequestId(), new CacheException("No query result found for request: " + req)); else if (qryResults.cancelled()) - sendError(node, req.queryRequestId(), new QueryCancelledException()); + sendError(node, req.queryRequestId(), new QueryCancelledException( + String.format("The query was cancelled while executing [queryId=%d, nodeId=%s, requestId=%s, reason=%s]", + req.query(), + node.id(), + req.queryRequestId(), + "Cancellation in query result" + ) + )); else { MapQueryLazyWorker lazyWorker = qryResults.lazyWorker(); @@ -1272,7 +1305,14 @@ else if (qryResults.cancelled()) JdbcSQLException sqlEx = X.cause(e, JdbcSQLException.class); if (sqlEx != null && sqlEx.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED) - sendError(node, qryResults.queryRequestId(), new QueryCancelledException()); + sendError(node, qryResults.queryRequestId(), new QueryCancelledException( + String.format("The query was cancelled while executing [queryId=%d, nodeId=%s, requestId=%s, reason=%s]", + req.query(), + node.id(), + req.queryRequestId(), + "Cancellation during lazy execution" + ) + )); else throw e; } 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 ecf0343d8bae2..48a7c270f7942 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 @@ -291,7 +291,7 @@ private void fail(ReduceQueryRun r, UUID nodeId, String msg, byte failCode) { if (failCode == GridQueryFailResponse.CANCELLED_BY_ORIGINATOR) { e = new CacheException("Failed to execute map query on remote node [nodeId=" + nodeId + - ", errMsg=" + msg + ']', new QueryCancelledException()); + ", reason=" + msg + ']', new QueryCancelledException()); } else { e = new CacheException("Failed to execute map query on remote node [nodeId=" + nodeId + @@ -950,8 +950,16 @@ else if (mvccTracker != null) if (e instanceof CacheException) { if (wasCancelled((CacheException)e)) - resEx = new CacheException("Failed to run reduce query locally.", - new QueryCancelledException()); + resEx = new CacheException( + "Failed to run reduce query locally.", + new QueryCancelledException( + String.format("The query was cancelled [query=%s, localNodeId=%s, reason=%s]", + qry.originalSql(), + ctx.localNodeId(), + e.getMessage() + ) + ) + ); else resEx = (CacheException)e; } @@ -965,7 +973,14 @@ else if (mvccTracker != null) Throwable cause = e; - if (e instanceof IgniteCheckedException) { + if (e instanceof QueryCancelledException) { + cause = new QueryCancelledException(String.format("The query was cancelled [query=%s, localNodeId=%s, reason=%s]", + qry.originalSql(), + ctx.localNodeId(), + e.getMessage() + )); + } + else if (e instanceof IgniteCheckedException) { Throwable disconnectedErr = ((IgniteCheckedException)e).getCause(IgniteClientDisconnectedException.class); @@ -1111,7 +1126,7 @@ public UpdateResult update( for (ClusterNode n : nodes) { if (!n.version().greaterThanEqual(2, 3, 0)) { log.warning("Server-side DML optimization is skipped because map node does not support it. " + - "Falling back to normal DML [node=" + n.id() + ", v=" + n.version() + "]."); + "Falling back to normal DML. [node=" + n.id() + ", v=" + n.version() + "]."); return null; } @@ -1201,7 +1216,7 @@ private void onDmlResponse(final ClusterNode node, GridH2DmlResponse msg) { DistributedUpdateRun r = updRuns.get(reqId); if (r == null) { - U.warn(log, "Unexpected dml response (will ignore) [localNodeId=" + ctx.localNodeId() + ", nodeId=" + + U.warn(log, "Unexpected dml response (will ignore). [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", msg=" + msg.toString() + ']'); return; @@ -1210,7 +1225,7 @@ private void onDmlResponse(final ClusterNode node, GridH2DmlResponse msg) { r.handleResponse(node.id(), msg); } catch (Exception e) { - U.error(log, "Error in dml response processing [localNodeId=" + ctx.localNodeId() + ", nodeId=" + + U.error(log, "Error in dml response processing. [localNodeId=" + ctx.localNodeId() + ", nodeId=" + node.id() + ", msg=" + msg.toString() + ']', e); } } @@ -1269,8 +1284,16 @@ public void releaseRemoteResources(Collection nodes, ReduceQueryRun } } - r.setStateOnException(ctx.localNodeId(), - new CacheException("Query is canceled.", new QueryCancelledException())); + r.setStateOnException( + ctx.localNodeId(), + new CacheException(new QueryCancelledException( + String.format("The query was cancelled while executing [query=%s, localNodeId=%s, reason=%s]", + r.queryInfo().query(), + ctx.localNodeId(), + "Cancelled by client" + )) + ) + ); if (!runs.remove(qryReqId, r)) U.warn(log, "Query run was already removed: " + qryReqId); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 35e8796c9e715..67b38a68306f8 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -135,15 +135,15 @@ public void testRemoteQueryWithMergeTableTimeout1() throws Exception { * @throws Exception If failed. */ public void testRemoteQueryExecutionCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false, - CANCELLED_BY_CLIENT); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 25, TimeUnit.MICROSECONDS, false, + "reason=The query was cancelled before initialization."); } /** * @throws Exception If failed. */ public void testRemoteQueryExecutionCancel1() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 10, TimeUnit.MILLISECONDS, false, + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 15, TimeUnit.MILLISECONDS, false, CANCELLED_BY_CLIENT); } @@ -151,7 +151,7 @@ public void testRemoteQueryExecutionCancel1() throws Exception { * @throws Exception If failed. */ public void testRemoteQueryExecutionCancel2() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false, + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 25, TimeUnit.MILLISECONDS, false, CANCELLED_BY_CLIENT); } @@ -310,6 +310,11 @@ private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUni assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException); assertTrue("Cause message " + ex.getCause().getMessage(), ex.getCause().getMessage().contains(cause)); + + cursor.close(); + } + catch (Throwable t){ + fail(t.getClass()+" "+t.getMessage()); } finally { // Give some time to clean up. From 15751b9dfc8c3297be0f218fd686371ca22744f3 Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 20 Nov 2018 13:34:30 +0300 Subject: [PATCH 95/95] Merge. --- .../h2/twostep/GridMapQueryExecutor.java | 10 ++++++---- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 20 +------------------ 2 files changed, 7 insertions(+), 23 deletions(-) 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 870d9268ee6cb..30b3ddfa301e7 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 @@ -98,8 +98,10 @@ import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.apache.ignite.thread.IgniteThread; +import org.h2.api.ErrorCode; import org.h2.command.Prepared; import org.h2.jdbc.JdbcResultSet; +import org.h2.jdbc.JdbcSQLException; import org.h2.value.Value; import org.jetbrains.annotations.Nullable; @@ -986,8 +988,8 @@ private void onQueryRequest0( qr.addResult(qryIdx, qry, node.id(), rs, params); - if (qryResults.cancelled()) { - qryResults.result(qryIdx).close(); + if (qr.cancelled()) { + qr.result(qryIdx).close(); throw new QueryCancelledException( String.format("The query was cancelled while executing [query=%s, localNodeId=%s, reason=%s, timeout=%s ms]", @@ -1320,13 +1322,13 @@ else if (qr.cancelled()) lazyWorker.submit(new Runnable() { @Override public void run() { try { - sendNextPage(nodeRess, node, qryResults, req.query(), req.segmentId(), req.pageSize(), false); + sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize(), false); } catch (Throwable e) { JdbcSQLException sqlEx = X.cause(e, JdbcSQLException.class); if (sqlEx != null && sqlEx.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED) - sendError(node, qryResults.queryRequestId(), new QueryCancelledException( + sendError(node, qr.queryRequestId(), new QueryCancelledException( String.format("The query was cancelled while executing [queryId=%d, nodeId=%s, requestId=%s, reason=%s]", req.query(), node.id(), diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 67b38a68306f8..41ddab4ea0829 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -163,14 +163,6 @@ public void testRemoteQueryExecutionCancel3() throws Exception { CANCELLED_BY_CLIENT); } - /** - * @throws Exception If failed. - */ - public void testRemoteQueryWithMergeTableCancel0() throws Exception { - testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false, - CANCELLED_BY_CLIENT); - } - /** * Query with far less complex sql and expected to be executed faster than timeout * @@ -198,22 +190,12 @@ public void testRemoteQueryWithMergeTableCancel3() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false, null); } - /** - * Query possibly could be executed faster than timeout. - * - * @throws Exception If failed. - */ - public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { - testQueryCancel(2*CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false, - CANCELLED_BY_CLIENT); - } - /** * Query with far less complex sql and expected to be executed faster than timeout. * * @throws Exception If failed. */ - public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { + public void testRemoteQueryWithoutMergeTableCancel() throws Exception { testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false, null); }